mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
filer: cross cluster synchronization
This commit is contained in:
parent
4fc0bd1a81
commit
387ab6796f
|
@ -58,6 +58,12 @@ service SeaweedFiler {
|
|||
rpc LocateBroker (LocateBrokerRequest) returns (LocateBrokerResponse) {
|
||||
}
|
||||
|
||||
rpc KvGet (KvGetRequest) returns (KvGetResponse) {
|
||||
}
|
||||
|
||||
rpc KvPut (KvPutRequest) returns (KvPutResponse) {
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////
|
||||
|
@ -308,3 +314,19 @@ message LocateBrokerResponse {
|
|||
}
|
||||
repeated Resource resources = 2;
|
||||
}
|
||||
|
||||
// Key-Value operations
|
||||
message KvGetRequest {
|
||||
bytes key = 1;
|
||||
}
|
||||
message KvGetResponse {
|
||||
bytes value = 1;
|
||||
string error = 2;
|
||||
}
|
||||
message KvPutRequest {
|
||||
bytes key = 1;
|
||||
bytes value = 2;
|
||||
}
|
||||
message KvPutResponse {
|
||||
string error = 1;
|
||||
}
|
||||
|
|
|
@ -16,6 +16,7 @@ var Commands = []*Command{
|
|||
cmdExport,
|
||||
cmdFiler,
|
||||
cmdFilerReplicate,
|
||||
cmdFilerSynchronize,
|
||||
cmdFix,
|
||||
cmdMaster,
|
||||
cmdMount,
|
||||
|
|
|
@ -72,7 +72,7 @@ var cmdCopy = &Command{
|
|||
|
||||
If "maxMB" is set to a positive number, files larger than it would be split into chunks.
|
||||
|
||||
`,
|
||||
`,
|
||||
}
|
||||
|
||||
func runCopy(cmd *Command, args []string) bool {
|
||||
|
|
329
weed/command/filer_sync.go
Normal file
329
weed/command/filer_sync.go
Normal file
|
@ -0,0 +1,329 @@
|
|||
package command
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"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"
|
||||
"github.com/chrislusf/seaweedfs/weed/replication/sink/filersink"
|
||||
"github.com/chrislusf/seaweedfs/weed/replication/source"
|
||||
"github.com/chrislusf/seaweedfs/weed/security"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
"google.golang.org/grpc"
|
||||
"io"
|
||||
"strings"
|
||||
"time"
|
||||
)
|
||||
|
||||
type SyncOptions struct {
|
||||
isActivePassive *bool
|
||||
filerA *string
|
||||
filerB *string
|
||||
aPath *string
|
||||
bPath *string
|
||||
aReplication *string
|
||||
bReplication *string
|
||||
aCollection *string
|
||||
bCollection *string
|
||||
aTtlSec *int
|
||||
bTtlSec *int
|
||||
aDebug *bool
|
||||
bDebug *bool
|
||||
}
|
||||
|
||||
var (
|
||||
syncOptions SyncOptions
|
||||
)
|
||||
|
||||
func init() {
|
||||
cmdFilerSynchronize.Run = runFilerSynchronize // break init cycle
|
||||
syncOptions.isActivePassive = cmdFilerSynchronize.Flag.Bool("isActivePassive", false, "one directional follow if true")
|
||||
syncOptions.filerA = cmdFilerSynchronize.Flag.String("a", "", "filer A in one SeaweedFS cluster")
|
||||
syncOptions.filerB = cmdFilerSynchronize.Flag.String("b", "", "filer B in the other SeaweedFS cluster")
|
||||
syncOptions.aPath = cmdFilerSynchronize.Flag.String("a.path", "/", "directory to sync on filer A")
|
||||
syncOptions.bPath = cmdFilerSynchronize.Flag.String("b.path", "/", "directory to sync on filer B")
|
||||
syncOptions.aReplication = cmdFilerSynchronize.Flag.String("a.replication", "", "replication on filer A")
|
||||
syncOptions.bReplication = cmdFilerSynchronize.Flag.String("b.replication", "", "replication on filer B")
|
||||
syncOptions.aCollection = cmdFilerSynchronize.Flag.String("a.collection", "", "collection on filer A")
|
||||
syncOptions.bCollection = cmdFilerSynchronize.Flag.String("b.collection", "", "collection on filer B")
|
||||
syncOptions.aTtlSec = cmdFilerSynchronize.Flag.Int("a.ttlSec", 0, "ttl in seconds on filer A")
|
||||
syncOptions.bTtlSec = cmdFilerSynchronize.Flag.Int("b.ttlSec", 0, "ttl in seconds on filer B")
|
||||
syncOptions.aDebug = cmdFilerSynchronize.Flag.Bool("a.debug", false, "debug mode to print out filer A received files")
|
||||
syncOptions.bDebug = cmdFilerSynchronize.Flag.Bool("b.debug", false, "debug mode to print out filer B received files")
|
||||
}
|
||||
|
||||
var cmdFilerSynchronize = &Command{
|
||||
UsageLine: "filer.sync -a=<oneFilerHost>:<oneFilerPort> -b=<otherFilerHost>:<otherFilerPort>",
|
||||
Short: "continuously synchronize between two active-active or active-passive SeaweedFS clusters",
|
||||
Long: `continuously synchronize file changes between two active-active or active-passive filers
|
||||
|
||||
filer.sync listens on filer notifications. If any file is updated, it will fetch the updated content,
|
||||
and write to the other destination. Different from filer.replicate:
|
||||
|
||||
* filer.sync only works between two filers.
|
||||
* filer.sync does not need any special message queue setup.
|
||||
* filer.sync supports both active-active and active-passive modes.
|
||||
|
||||
If restarted, the synchronization will resume from the previous checkpoints, persisted every minute.
|
||||
|
||||
`,
|
||||
}
|
||||
|
||||
func runFilerSynchronize(cmd *Command, args []string) bool {
|
||||
|
||||
grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.client")
|
||||
|
||||
go func() {
|
||||
for {
|
||||
err := doSubscribeFilerMetaChanges(grpcDialOption, *syncOptions.filerA, *syncOptions.aPath, *syncOptions.filerB,
|
||||
*syncOptions.bPath, *syncOptions.bReplication, *syncOptions.bCollection, *syncOptions.bTtlSec, *syncOptions.bDebug)
|
||||
if err != nil {
|
||||
glog.Errorf("sync from %s to %s: %v", *syncOptions.filerA, *syncOptions.filerB, err)
|
||||
time.Sleep(1747 * time.Millisecond)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
if !*syncOptions.isActivePassive {
|
||||
go func() {
|
||||
for {
|
||||
err := doSubscribeFilerMetaChanges(grpcDialOption, *syncOptions.filerB, *syncOptions.bPath, *syncOptions.filerA,
|
||||
*syncOptions.aPath, *syncOptions.aReplication, *syncOptions.aCollection, *syncOptions.aTtlSec, *syncOptions.aDebug)
|
||||
if err != nil {
|
||||
glog.Errorf("sync from %s to %s: %v", *syncOptions.filerB, *syncOptions.filerA, err)
|
||||
time.Sleep(2147 * time.Millisecond)
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
select {}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
func doSubscribeFilerMetaChanges(grpcDialOption grpc.DialOption, sourceFiler, sourcePath, targetFiler, targetPath string,
|
||||
replicationStr, collection string, ttlSec int, debug bool) error {
|
||||
|
||||
// read source filer signature
|
||||
sourceFilerSignature, sourceErr := replication.ReadFilerSignature(grpcDialOption, sourceFiler)
|
||||
if sourceErr != nil {
|
||||
return sourceErr
|
||||
}
|
||||
// read target filer signature
|
||||
targetFilerSignature, targetErr := replication.ReadFilerSignature(grpcDialOption, targetFiler)
|
||||
if targetErr != nil {
|
||||
return targetErr
|
||||
}
|
||||
|
||||
// if first time, start from now
|
||||
// if has previously synced, resume from that point of time
|
||||
sourceFilerOffsetTsNs, err := readSyncOffset(grpcDialOption, targetFiler, sourceFilerSignature)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
glog.V(0).Infof("start sync %s(%d) => %s(%d) from %v(%d)", sourceFiler, sourceFilerSignature, targetFiler, targetFilerSignature, time.Unix(0, sourceFilerOffsetTsNs), sourceFilerOffsetTsNs)
|
||||
|
||||
// create filer sink
|
||||
filerSource := &source.FilerSource{}
|
||||
filerSource.DoInitialize(pb.ServerToGrpcAddress(sourceFiler), sourcePath)
|
||||
filerSink := &filersink.FilerSink{}
|
||||
filerSink.DoInitialize(pb.ServerToGrpcAddress(targetFiler), targetPath, replicationStr, collection, ttlSec, grpcDialOption)
|
||||
filerSink.SetSourceFiler(filerSource)
|
||||
|
||||
processEventFn := func(resp *filer_pb.SubscribeMetadataResponse) error {
|
||||
message := resp.EventNotification
|
||||
|
||||
var sourceOldKey, sourceNewKey util.FullPath
|
||||
if message.OldEntry != nil {
|
||||
sourceOldKey = util.FullPath(resp.Directory).Child(message.OldEntry.Name)
|
||||
}
|
||||
if message.NewEntry != nil {
|
||||
sourceNewKey = util.FullPath(message.NewParentPath).Child(message.NewEntry.Name)
|
||||
}
|
||||
|
||||
for _, sig := range message.Signatures {
|
||||
if sig == targetFilerSignature && targetFilerSignature != 0 {
|
||||
fmt.Printf("%s skipping %s change to %v\n", targetFiler, sourceFiler, message)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
if debug {
|
||||
fmt.Printf("%s check %s change %s,%s sig %v, target sig: %v\n", targetFiler, sourceFiler, sourceOldKey, sourceNewKey, message.Signatures, targetFilerSignature)
|
||||
}
|
||||
|
||||
if !strings.HasPrefix(resp.Directory, sourcePath) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// handle deletions
|
||||
if message.OldEntry != nil && message.NewEntry == nil {
|
||||
if !strings.HasPrefix(string(sourceOldKey), sourcePath) {
|
||||
return nil
|
||||
}
|
||||
key := util.Join(targetPath, string(sourceOldKey)[len(sourcePath):])
|
||||
return filerSink.DeleteEntry(key, message.OldEntry.IsDirectory, message.DeleteChunks, message.Signatures)
|
||||
}
|
||||
|
||||
// handle new entries
|
||||
if message.OldEntry == nil && message.NewEntry != nil {
|
||||
if !strings.HasPrefix(string(sourceNewKey), sourcePath) {
|
||||
return nil
|
||||
}
|
||||
key := util.Join(targetPath, string(sourceNewKey)[len(sourcePath):])
|
||||
return filerSink.CreateEntry(key, message.NewEntry, message.Signatures)
|
||||
}
|
||||
|
||||
// this is something special?
|
||||
if message.OldEntry == nil && message.NewEntry == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// handle updates
|
||||
if strings.HasPrefix(string(sourceOldKey), sourcePath) {
|
||||
// old key is in the watched directory
|
||||
if strings.HasPrefix(string(sourceNewKey), sourcePath) {
|
||||
// new key is also in the watched directory
|
||||
oldKey := util.Join(targetPath, string(sourceOldKey)[len(sourcePath):])
|
||||
message.NewParentPath = util.Join(targetPath, message.NewParentPath[len(sourcePath):])
|
||||
foundExisting, err := filerSink.UpdateEntry(string(oldKey), message.OldEntry, message.NewParentPath, message.NewEntry, message.DeleteChunks, message.Signatures)
|
||||
if foundExisting {
|
||||
return err
|
||||
}
|
||||
|
||||
// not able to find old entry
|
||||
if err = filerSink.DeleteEntry(string(oldKey), message.OldEntry.IsDirectory, false, message.Signatures); err != nil {
|
||||
return fmt.Errorf("delete old entry %v: %v", oldKey, err)
|
||||
}
|
||||
|
||||
// create the new entry
|
||||
newKey := util.Join(targetPath, string(sourceNewKey)[len(sourcePath):])
|
||||
return filerSink.CreateEntry(newKey, message.NewEntry, message.Signatures)
|
||||
|
||||
} else {
|
||||
// new key is outside of the watched directory
|
||||
key := util.Join(targetPath, string(sourceOldKey)[len(sourcePath):])
|
||||
return filerSink.DeleteEntry(key, message.OldEntry.IsDirectory, message.DeleteChunks, message.Signatures)
|
||||
}
|
||||
} else {
|
||||
// old key is outside of the watched directory
|
||||
if strings.HasPrefix(string(sourceNewKey), sourcePath) {
|
||||
// new key is in the watched directory
|
||||
key := util.Join(targetPath, string(sourceNewKey)[len(sourcePath):])
|
||||
return filerSink.CreateEntry(key, message.NewEntry, message.Signatures)
|
||||
} else {
|
||||
// new key is also outside of the watched directory
|
||||
// skip
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
return 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 => %s progressed to %v %0.2f/sec", sourceFiler, targetFiler, time.Unix(0, resp.TsNs), float64(counter)/float64(3))
|
||||
counter = 0
|
||||
lastWriteTime = time.Now()
|
||||
if err := writeSyncOffset(grpcDialOption, targetFiler, sourceFilerSignature, resp.TsNs); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
const (
|
||||
SyncKeyPrefix = "sync."
|
||||
)
|
||||
|
||||
func readSyncOffset(grpcDialOption grpc.DialOption, filer string, filerSignature int32) (lastOffsetTsNs int64, readErr error) {
|
||||
|
||||
readErr = pb.WithFilerClient(filer, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
syncKey := []byte(SyncKeyPrefix + "____")
|
||||
util.Uint32toBytes(syncKey[len(SyncKeyPrefix):len(SyncKeyPrefix)+4], uint32(filerSignature))
|
||||
|
||||
resp, err := client.KvGet(context.Background(), &filer_pb.KvGetRequest{Key: syncKey})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if len(resp.Error) != 0 {
|
||||
return errors.New(resp.Error)
|
||||
}
|
||||
if len(resp.Value) < 8 {
|
||||
return nil
|
||||
}
|
||||
|
||||
lastOffsetTsNs = int64(util.BytesToUint64(resp.Value))
|
||||
|
||||
return nil
|
||||
})
|
||||
|
||||
return
|
||||
|
||||
}
|
||||
|
||||
func writeSyncOffset(grpcDialOption grpc.DialOption, filer string, filerSignature int32, offsetTsNs int64) error {
|
||||
return pb.WithFilerClient(filer, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
|
||||
syncKey := []byte(SyncKeyPrefix + "____")
|
||||
util.Uint32toBytes(syncKey[len(SyncKeyPrefix):len(SyncKeyPrefix)+4], uint32(filerSignature))
|
||||
|
||||
valueBuf := make([]byte, 8)
|
||||
util.Uint64toBytes(valueBuf, uint64(offsetTsNs))
|
||||
|
||||
resp, err := client.KvPut(context.Background(), &filer_pb.KvPutRequest{
|
||||
Key: syncKey,
|
||||
Value: valueBuf,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if len(resp.Error) != 0 {
|
||||
return errors.New(resp.Error)
|
||||
}
|
||||
|
||||
return nil
|
||||
|
||||
})
|
||||
|
||||
}
|
|
@ -78,7 +78,10 @@ func runWatch(cmd *Command, args []string) bool {
|
|||
|
||||
watchErr := pb.WithFilerClient(*watchFiler, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
|
||||
stream, err := client.SubscribeMetadata(context.Background(), &filer_pb.SubscribeMetadataRequest{
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: "watch",
|
||||
PathPrefix: *watchTarget,
|
||||
SinceNs: time.Now().Add(-*watchStart).UnixNano(),
|
||||
|
|
|
@ -226,6 +226,11 @@ func NonOverlappingVisibleIntervals(lookupFileIdFn LookupFileIdFunctionType, chu
|
|||
|
||||
sort.Slice(chunks, func(i, j int) bool {
|
||||
if chunks[i].Mtime == chunks[j].Mtime {
|
||||
filer_pb.EnsureFid(chunks[i])
|
||||
filer_pb.EnsureFid(chunks[j])
|
||||
if chunks[i].Fid == nil || chunks[j].Fid == nil {
|
||||
return true
|
||||
}
|
||||
return chunks[i].Fid.FileKey < chunks[j].Fid.FileKey
|
||||
}
|
||||
return chunks[i].Mtime < chunks[j].Mtime // keep this to make tests run
|
||||
|
|
|
@ -27,7 +27,7 @@ func (f *Filer) DeleteEntryMetaAndData(ctx context.Context, p util.FullPath, isR
|
|||
if entry.IsDirectory() {
|
||||
// delete the folder children, not including the folder itself
|
||||
var dirChunks []*filer_pb.FileChunk
|
||||
dirChunks, err = f.doBatchDeleteFolderMetaAndData(ctx, entry, isRecursive, ignoreRecursiveError, shouldDeleteChunks && !isCollection, isFromOtherCluster)
|
||||
dirChunks, err = f.doBatchDeleteFolderMetaAndData(ctx, entry, isRecursive, ignoreRecursiveError, shouldDeleteChunks && !isCollection, isFromOtherCluster, signatures)
|
||||
if err != nil {
|
||||
glog.V(0).Infof("delete directory %s: %v", p, err)
|
||||
return fmt.Errorf("delete directory %s: %v", p, err)
|
||||
|
@ -53,7 +53,7 @@ func (f *Filer) DeleteEntryMetaAndData(ctx context.Context, p util.FullPath, isR
|
|||
return nil
|
||||
}
|
||||
|
||||
func (f *Filer) doBatchDeleteFolderMetaAndData(ctx context.Context, entry *Entry, isRecursive, ignoreRecursiveError, shouldDeleteChunks, isFromOtherCluster bool) (chunks []*filer_pb.FileChunk, err error) {
|
||||
func (f *Filer) doBatchDeleteFolderMetaAndData(ctx context.Context, entry *Entry, isRecursive, ignoreRecursiveError, shouldDeleteChunks, isFromOtherCluster bool, signatures []int32) (chunks []*filer_pb.FileChunk, err error) {
|
||||
|
||||
lastFileName := ""
|
||||
includeLastFile := false
|
||||
|
@ -73,7 +73,7 @@ func (f *Filer) doBatchDeleteFolderMetaAndData(ctx context.Context, entry *Entry
|
|||
lastFileName = sub.Name()
|
||||
var dirChunks []*filer_pb.FileChunk
|
||||
if sub.IsDirectory() {
|
||||
dirChunks, err = f.doBatchDeleteFolderMetaAndData(ctx, sub, isRecursive, ignoreRecursiveError, shouldDeleteChunks, false)
|
||||
dirChunks, err = f.doBatchDeleteFolderMetaAndData(ctx, sub, isRecursive, ignoreRecursiveError, shouldDeleteChunks, false, nil)
|
||||
chunks = append(chunks, dirChunks...)
|
||||
} else {
|
||||
f.NotifyUpdateEvent(ctx, sub, nil, shouldDeleteChunks, isFromOtherCluster, nil)
|
||||
|
@ -95,7 +95,7 @@ func (f *Filer) doBatchDeleteFolderMetaAndData(ctx context.Context, entry *Entry
|
|||
return nil, fmt.Errorf("filer store delete: %v", storeDeletionErr)
|
||||
}
|
||||
|
||||
f.NotifyUpdateEvent(ctx, entry, nil, shouldDeleteChunks, isFromOtherCluster, nil)
|
||||
f.NotifyUpdateEvent(ctx, entry, nil, shouldDeleteChunks, isFromOtherCluster, signatures)
|
||||
|
||||
return chunks, nil
|
||||
}
|
||||
|
|
|
@ -30,6 +30,15 @@ func (f *Filer) NotifyUpdateEvent(ctx context.Context, oldEntry, newEntry *Entry
|
|||
if strings.HasPrefix(fullpath, SystemLogDir) {
|
||||
return
|
||||
}
|
||||
foundSelf := false
|
||||
for _, sig := range signatures {
|
||||
if sig == f.Signature {
|
||||
foundSelf = true
|
||||
}
|
||||
}
|
||||
if !foundSelf {
|
||||
signatures = append(signatures, f.Signature)
|
||||
}
|
||||
|
||||
newParentPath := ""
|
||||
if newEntry != nil {
|
||||
|
@ -41,7 +50,7 @@ func (f *Filer) NotifyUpdateEvent(ctx context.Context, oldEntry, newEntry *Entry
|
|||
DeleteChunks: deleteChunks,
|
||||
NewParentPath: newParentPath,
|
||||
IsFromOtherCluster: isFromOtherCluster,
|
||||
Signatures: append(signatures, f.Signature),
|
||||
Signatures: signatures,
|
||||
}
|
||||
|
||||
if notification.Queue != nil {
|
||||
|
|
|
@ -328,7 +328,7 @@ func (dir *Dir) removeOneFile(req *fuse.RemoveRequest) error {
|
|||
|
||||
// first, ensure the filer store can correctly delete
|
||||
glog.V(3).Infof("remove file: %v", req)
|
||||
err = filer_pb.Remove(dir.wfs, dir.FullPath(), req.Name, false, false, false, false, dir.wfs.signature)
|
||||
err = filer_pb.Remove(dir.wfs, dir.FullPath(), req.Name, false, false, false, false, []int32{dir.wfs.signature})
|
||||
if err != nil {
|
||||
glog.V(3).Infof("not found remove file %s/%s: %v", dir.FullPath(), req.Name, err)
|
||||
return fuse.ENOENT
|
||||
|
@ -348,7 +348,7 @@ func (dir *Dir) removeOneFile(req *fuse.RemoveRequest) error {
|
|||
func (dir *Dir) removeFolder(req *fuse.RemoveRequest) error {
|
||||
|
||||
glog.V(3).Infof("remove directory entry: %v", req)
|
||||
err := filer_pb.Remove(dir.wfs, dir.FullPath(), req.Name, true, false, false, false, dir.wfs.signature)
|
||||
err := filer_pb.Remove(dir.wfs, dir.FullPath(), req.Name, true, false, false, false, []int32{dir.wfs.signature})
|
||||
if err != nil {
|
||||
glog.V(0).Infof("remove %s/%s: %v", dir.FullPath(), req.Name, err)
|
||||
if strings.Contains(err.Error(), "non-empty") {
|
||||
|
|
|
@ -44,7 +44,9 @@ func SubscribeMetaEvents(mc *MetaCache, selfSignature int32, client filer_pb.Fil
|
|||
|
||||
for {
|
||||
err := client.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
stream, err := client.SubscribeMetadata(context.Background(), &filer_pb.SubscribeMetadataRequest{
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: "mount",
|
||||
PathPrefix: dir,
|
||||
SinceNs: lastTsNs,
|
||||
|
@ -71,7 +73,7 @@ func SubscribeMetaEvents(mc *MetaCache, selfSignature int32, client filer_pb.Fil
|
|||
})
|
||||
if err != nil {
|
||||
glog.Errorf("subscribing filer meta change: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@ func (broker *MessageBroker) DeleteTopic(c context.Context, request *messaging_p
|
|||
if exists, err := filer_pb.Exists(broker, dir, entry, true); err != nil {
|
||||
return nil, err
|
||||
} else if exists {
|
||||
err = filer_pb.Remove(broker, dir, entry, true, true, true, false, 0)
|
||||
err = filer_pb.Remove(broker, dir, entry, true, true, true, false, nil)
|
||||
}
|
||||
return resp, nil
|
||||
}
|
||||
|
|
|
@ -2,6 +2,7 @@ package broker
|
|||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/util/log_buffer"
|
||||
"io"
|
||||
"strings"
|
||||
"time"
|
||||
|
@ -113,12 +114,21 @@ func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_Subs
|
|||
|
||||
// fmt.Printf("subscriber %s read %d on disk log %v\n", subscriberId, messageCount, lastReadTime)
|
||||
|
||||
err = lock.logBuffer.LoopProcessLogData(lastReadTime, func() bool {
|
||||
for {
|
||||
lastReadTime, err = lock.logBuffer.LoopProcessLogData(lastReadTime, func() bool {
|
||||
lock.Mutex.Lock()
|
||||
lock.cond.Wait()
|
||||
lock.Mutex.Unlock()
|
||||
return isConnected
|
||||
}, eachLogEntryFn)
|
||||
if err != nil {
|
||||
glog.Errorf("processed to %v: %v", lastReadTime, err)
|
||||
time.Sleep(3127 * time.Millisecond)
|
||||
if err != log_buffer.ResumeError {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return err
|
||||
|
||||
|
|
|
@ -11,6 +11,7 @@ import (
|
|||
"net/http"
|
||||
"net/textproto"
|
||||
"path/filepath"
|
||||
"runtime/debug"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
|
@ -85,7 +86,7 @@ func doUpload(uploadUrl string, filename string, cipher bool, reader io.Reader,
|
|||
}
|
||||
|
||||
func retriedUploadData(uploadUrl string, filename string, cipher bool, data []byte, isInputCompressed bool, mtype string, pairMap map[string]string, jwt security.EncodedJwt) (uploadResult *UploadResult, err error) {
|
||||
for i := 0; i < 3; i++ {
|
||||
for i := 0; i < 1; i++ {
|
||||
uploadResult, err = doUploadData(uploadUrl, filename, cipher, data, isInputCompressed, mtype, pairMap, jwt)
|
||||
if err == nil {
|
||||
return
|
||||
|
@ -221,8 +222,9 @@ func upload_content(uploadUrl string, fillBufferFunction func(w io.Writer) error
|
|||
}
|
||||
resp, post_err := HttpClient.Do(req)
|
||||
if post_err != nil {
|
||||
glog.Errorf("upload to %v: %v", uploadUrl, post_err)
|
||||
return nil, fmt.Errorf("upload to %v: %v", uploadUrl, post_err)
|
||||
glog.Errorf("upload %s %d bytes to %v: %v", filename, originalDataSize, uploadUrl, post_err)
|
||||
debug.PrintStack()
|
||||
return nil, fmt.Errorf("upload %s %d bytes to %v: %v", filename, originalDataSize, uploadUrl, post_err)
|
||||
}
|
||||
defer util.CloseResponse(resp)
|
||||
|
||||
|
|
|
@ -58,6 +58,12 @@ service SeaweedFiler {
|
|||
rpc LocateBroker (LocateBrokerRequest) returns (LocateBrokerResponse) {
|
||||
}
|
||||
|
||||
rpc KvGet (KvGetRequest) returns (KvGetResponse) {
|
||||
}
|
||||
|
||||
rpc KvPut (KvPutRequest) returns (KvPutResponse) {
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////
|
||||
|
@ -308,3 +314,19 @@ message LocateBrokerResponse {
|
|||
}
|
||||
repeated Resource resources = 2;
|
||||
}
|
||||
|
||||
// Key-Value operations
|
||||
message KvGetRequest {
|
||||
bytes key = 1;
|
||||
}
|
||||
message KvGetResponse {
|
||||
bytes value = 1;
|
||||
string error = 2;
|
||||
}
|
||||
message KvPutRequest {
|
||||
bytes key = 1;
|
||||
bytes value = 2;
|
||||
}
|
||||
message KvPutResponse {
|
||||
string error = 1;
|
||||
}
|
||||
|
|
|
@ -2608,6 +2608,211 @@ func (x *LocateBrokerResponse) GetResources() []*LocateBrokerResponse_Resource {
|
|||
return nil
|
||||
}
|
||||
|
||||
// Key-Value operations
|
||||
type KvGetRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
|
||||
}
|
||||
|
||||
func (x *KvGetRequest) Reset() {
|
||||
*x = KvGetRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_filer_proto_msgTypes[40]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *KvGetRequest) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*KvGetRequest) ProtoMessage() {}
|
||||
|
||||
func (x *KvGetRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_filer_proto_msgTypes[40]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use KvGetRequest.ProtoReflect.Descriptor instead.
|
||||
func (*KvGetRequest) Descriptor() ([]byte, []int) {
|
||||
return file_filer_proto_rawDescGZIP(), []int{40}
|
||||
}
|
||||
|
||||
func (x *KvGetRequest) GetKey() []byte {
|
||||
if x != nil {
|
||||
return x.Key
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type KvGetResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Value []byte `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"`
|
||||
Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"`
|
||||
}
|
||||
|
||||
func (x *KvGetResponse) Reset() {
|
||||
*x = KvGetResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_filer_proto_msgTypes[41]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *KvGetResponse) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*KvGetResponse) ProtoMessage() {}
|
||||
|
||||
func (x *KvGetResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_filer_proto_msgTypes[41]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use KvGetResponse.ProtoReflect.Descriptor instead.
|
||||
func (*KvGetResponse) Descriptor() ([]byte, []int) {
|
||||
return file_filer_proto_rawDescGZIP(), []int{41}
|
||||
}
|
||||
|
||||
func (x *KvGetResponse) GetValue() []byte {
|
||||
if x != nil {
|
||||
return x.Value
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *KvGetResponse) GetError() string {
|
||||
if x != nil {
|
||||
return x.Error
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type KvPutRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
|
||||
Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
|
||||
}
|
||||
|
||||
func (x *KvPutRequest) Reset() {
|
||||
*x = KvPutRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_filer_proto_msgTypes[42]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *KvPutRequest) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*KvPutRequest) ProtoMessage() {}
|
||||
|
||||
func (x *KvPutRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_filer_proto_msgTypes[42]
|
||||
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 KvPutRequest.ProtoReflect.Descriptor instead.
|
||||
func (*KvPutRequest) Descriptor() ([]byte, []int) {
|
||||
return file_filer_proto_rawDescGZIP(), []int{42}
|
||||
}
|
||||
|
||||
func (x *KvPutRequest) GetKey() []byte {
|
||||
if x != nil {
|
||||
return x.Key
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *KvPutRequest) GetValue() []byte {
|
||||
if x != nil {
|
||||
return x.Value
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type KvPutResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"`
|
||||
}
|
||||
|
||||
func (x *KvPutResponse) Reset() {
|
||||
*x = KvPutResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_filer_proto_msgTypes[43]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *KvPutResponse) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*KvPutResponse) ProtoMessage() {}
|
||||
|
||||
func (x *KvPutResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_filer_proto_msgTypes[43]
|
||||
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 KvPutResponse.ProtoReflect.Descriptor instead.
|
||||
func (*KvPutResponse) Descriptor() ([]byte, []int) {
|
||||
return file_filer_proto_rawDescGZIP(), []int{43}
|
||||
}
|
||||
|
||||
func (x *KvPutResponse) GetError() string {
|
||||
if x != nil {
|
||||
return x.Error
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// if found, send the exact address
|
||||
// if not found, send the full list of existing brokers
|
||||
type LocateBrokerResponse_Resource struct {
|
||||
|
@ -2622,7 +2827,7 @@ type LocateBrokerResponse_Resource struct {
|
|||
func (x *LocateBrokerResponse_Resource) Reset() {
|
||||
*x = LocateBrokerResponse_Resource{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_filer_proto_msgTypes[42]
|
||||
mi := &file_filer_proto_msgTypes[46]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -2635,7 +2840,7 @@ func (x *LocateBrokerResponse_Resource) String() string {
|
|||
func (*LocateBrokerResponse_Resource) ProtoMessage() {}
|
||||
|
||||
func (x *LocateBrokerResponse_Resource) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_filer_proto_msgTypes[42]
|
||||
mi := &file_filer_proto_msgTypes[46]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -3001,102 +3206,121 @@ var file_filer_proto_rawDesc = []byte{
|
|||
0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x67, 0x72, 0x70, 0x63, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73,
|
||||
0x73, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f,
|
||||
0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 0x73,
|
||||
0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x32, 0x8d, 0x0b, 0x0a, 0x0c, 0x53,
|
||||
0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x12, 0x67, 0x0a, 0x14, 0x4c,
|
||||
0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e,
|
||||
0x74, 0x72, 0x79, 0x12, 0x25, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c,
|
||||
0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e,
|
||||
0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x66, 0x69, 0x6c,
|
||||
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65,
|
||||
0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
|
||||
0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72,
|
||||
0x69, 0x65, 0x73, 0x12, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c,
|
||||
0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
|
||||
0x74, 0x1a, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73,
|
||||
0x74, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
|
||||
0x22, 0x00, 0x30, 0x01, 0x12, 0x4c, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e,
|
||||
0x74, 0x72, 0x79, 0x12, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43,
|
||||
0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
|
||||
0x74, 0x1a, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65,
|
||||
0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
|
||||
0x22, 0x00, 0x12, 0x4c, 0x0a, 0x0b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72,
|
||||
0x79, 0x12, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64,
|
||||
0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x20, 0x0a, 0x0c, 0x4b, 0x76,
|
||||
0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65,
|
||||
0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0x3b, 0x0a, 0x0d,
|
||||
0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a,
|
||||
0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61,
|
||||
0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x36, 0x0a, 0x0c, 0x4b, 0x76, 0x50,
|
||||
0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79,
|
||||
0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76,
|
||||
0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75,
|
||||
0x65, 0x22, 0x25, 0x0a, 0x0d, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
|
||||
0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x32, 0x85, 0x0c, 0x0a, 0x0c, 0x53, 0x65, 0x61,
|
||||
0x77, 0x65, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x12, 0x67, 0x0a, 0x14, 0x4c, 0x6f, 0x6f,
|
||||
0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72,
|
||||
0x79, 0x12, 0x25, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f,
|
||||
0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72,
|
||||
0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72,
|
||||
0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74,
|
||||
0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
|
||||
0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65,
|
||||
0x73, 0x12, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73,
|
||||
0x74, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
|
||||
0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x45,
|
||||
0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
|
||||
0x30, 0x01, 0x12, 0x4c, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72,
|
||||
0x79, 0x12, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65,
|
||||
0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
|
||||
0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74,
|
||||
0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74,
|
||||
0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
|
||||
0x12, 0x52, 0x0a, 0x0d, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72,
|
||||
0x79, 0x12, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70,
|
||||
0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
|
||||
0x74, 0x1a, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70,
|
||||
0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
|
||||
0x73, 0x65, 0x22, 0x00, 0x12, 0x4c, 0x0a, 0x0b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e,
|
||||
0x74, 0x72, 0x79, 0x12, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44,
|
||||
0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
|
||||
0x74, 0x1a, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c,
|
||||
0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
|
||||
0x22, 0x00, 0x12, 0x5e, 0x0a, 0x11, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61,
|
||||
0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f,
|
||||
0x70, 0x62, 0x2e, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45,
|
||||
0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69,
|
||||
0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e,
|
||||
0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
|
||||
0x22, 0x00, 0x12, 0x4f, 0x0a, 0x0c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75,
|
||||
0x6d, 0x65, 0x12, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73,
|
||||
0x73, 0x69, 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
|
||||
0x74, 0x1a, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73,
|
||||
0x69, 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
|
||||
0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x0c, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c,
|
||||
0x75, 0x6d, 0x65, 0x12, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c,
|
||||
0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
|
||||
0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f,
|
||||
0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
|
||||
0x73, 0x65, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f,
|
||||
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72,
|
||||
0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
|
||||
0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x66, 0x69,
|
||||
0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c,
|
||||
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
|
||||
0x00, 0x12, 0x49, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12,
|
||||
0x1b, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x69,
|
||||
0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x66,
|
||||
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69,
|
||||
0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6a, 0x0a, 0x15,
|
||||
0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72,
|
||||
0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x26, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62,
|
||||
0x2e, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75,
|
||||
0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e,
|
||||
0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65,
|
||||
0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65,
|
||||
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x53, 0x75, 0x62, 0x73,
|
||||
0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x22, 0x2e,
|
||||
0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69,
|
||||
0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
|
||||
0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62,
|
||||
0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65,
|
||||
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x65, 0x0a, 0x16, 0x53, 0x75,
|
||||
0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x4d, 0x65, 0x74, 0x61,
|
||||
0x64, 0x61, 0x74, 0x61, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e,
|
||||
0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74,
|
||||
0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72,
|
||||
0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74,
|
||||
0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30,
|
||||
0x01, 0x12, 0x56, 0x0a, 0x0d, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,
|
||||
0x65, 0x64, 0x12, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x65,
|
||||
0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65,
|
||||
0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x65,
|
||||
0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f,
|
||||
0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x4f, 0x0a, 0x0c, 0x4c, 0x6f, 0x63,
|
||||
0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65,
|
||||
0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65,
|
||||
0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72,
|
||||
0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72,
|
||||
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x4f, 0x0a, 0x10, 0x73, 0x65,
|
||||
0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2e, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x42, 0x0a,
|
||||
0x46, 0x69, 0x6c, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68,
|
||||
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, 0x68, 0x72, 0x69, 0x73, 0x6c, 0x75, 0x73, 0x66,
|
||||
0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f,
|
||||
0x70, 0x62, 0x2f, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x33,
|
||||
0x12, 0x4c, 0x0a, 0x0b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12,
|
||||
0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74,
|
||||
0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e,
|
||||
0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45,
|
||||
0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52,
|
||||
0x0a, 0x0d, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12,
|
||||
0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e,
|
||||
0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
|
||||
0x1f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e,
|
||||
0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
|
||||
0x22, 0x00, 0x12, 0x4c, 0x0a, 0x0b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72,
|
||||
0x79, 0x12, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c,
|
||||
0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
|
||||
0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74,
|
||||
0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
|
||||
0x12, 0x5e, 0x0a, 0x11, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65,
|
||||
0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62,
|
||||
0x2e, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74,
|
||||
0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65,
|
||||
0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d,
|
||||
0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
|
||||
0x12, 0x4f, 0x0a, 0x0c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
|
||||
0x12, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69,
|
||||
0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
|
||||
0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67,
|
||||
0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
|
||||
0x00, 0x12, 0x4f, 0x0a, 0x0c, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
|
||||
0x65, 0x12, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f,
|
||||
0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x1a, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b,
|
||||
0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
|
||||
0x22, 0x00, 0x12, 0x5b, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c,
|
||||
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70,
|
||||
0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
||||
0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65,
|
||||
0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65,
|
||||
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
|
||||
0x49, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x1b, 0x2e,
|
||||
0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74,
|
||||
0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x66, 0x69, 0x6c,
|
||||
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73,
|
||||
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6a, 0x0a, 0x15, 0x47, 0x65,
|
||||
0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74,
|
||||
0x69, 0x6f, 0x6e, 0x12, 0x26, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47,
|
||||
0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61,
|
||||
0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x66, 0x69,
|
||||
0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43,
|
||||
0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70,
|
||||
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72,
|
||||
0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x22, 0x2e, 0x66, 0x69,
|
||||
0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
|
||||
0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
|
||||
0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63,
|
||||
0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70,
|
||||
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x65, 0x0a, 0x16, 0x53, 0x75, 0x62, 0x73,
|
||||
0x63, 0x72, 0x69, 0x62, 0x65, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61,
|
||||
0x74, 0x61, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75,
|
||||
0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70,
|
||||
0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64,
|
||||
0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12,
|
||||
0x56, 0x0a, 0x0d, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64,
|
||||
0x12, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x65, 0x65, 0x70,
|
||||
0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x1a, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x65, 0x65, 0x70,
|
||||
0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
|
||||
0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x4f, 0x0a, 0x0c, 0x4c, 0x6f, 0x63, 0x61, 0x74,
|
||||
0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f,
|
||||
0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70,
|
||||
0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x52, 0x65,
|
||||
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x4b, 0x76, 0x47, 0x65,
|
||||
0x74, 0x12, 0x16, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x47,
|
||||
0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x66, 0x69, 0x6c, 0x65,
|
||||
0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
|
||||
0x73, 0x65, 0x22, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x12, 0x16, 0x2e,
|
||||
0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65,
|
||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62,
|
||||
0x2e, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
|
||||
0x42, 0x4f, 0x0a, 0x10, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2e, 0x63, 0x6c,
|
||||
0x69, 0x65, 0x6e, 0x74, 0x42, 0x0a, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, 0x68, 0x72,
|
||||
0x69, 0x73, 0x6c, 0x75, 0x73, 0x66, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73,
|
||||
0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f, 0x70, 0x62, 0x2f, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70,
|
||||
0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
@ -3111,7 +3335,7 @@ func file_filer_proto_rawDescGZIP() []byte {
|
|||
return file_filer_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_filer_proto_msgTypes = make([]protoimpl.MessageInfo, 43)
|
||||
var file_filer_proto_msgTypes = make([]protoimpl.MessageInfo, 47)
|
||||
var file_filer_proto_goTypes = []interface{}{
|
||||
(*LookupDirectoryEntryRequest)(nil), // 0: filer_pb.LookupDirectoryEntryRequest
|
||||
(*LookupDirectoryEntryResponse)(nil), // 1: filer_pb.LookupDirectoryEntryResponse
|
||||
|
@ -3153,16 +3377,20 @@ var file_filer_proto_goTypes = []interface{}{
|
|||
(*KeepConnectedResponse)(nil), // 37: filer_pb.KeepConnectedResponse
|
||||
(*LocateBrokerRequest)(nil), // 38: filer_pb.LocateBrokerRequest
|
||||
(*LocateBrokerResponse)(nil), // 39: filer_pb.LocateBrokerResponse
|
||||
nil, // 40: filer_pb.Entry.ExtendedEntry
|
||||
nil, // 41: filer_pb.LookupVolumeResponse.LocationsMapEntry
|
||||
(*LocateBrokerResponse_Resource)(nil), // 42: filer_pb.LocateBrokerResponse.Resource
|
||||
(*KvGetRequest)(nil), // 40: filer_pb.KvGetRequest
|
||||
(*KvGetResponse)(nil), // 41: filer_pb.KvGetResponse
|
||||
(*KvPutRequest)(nil), // 42: filer_pb.KvPutRequest
|
||||
(*KvPutResponse)(nil), // 43: filer_pb.KvPutResponse
|
||||
nil, // 44: filer_pb.Entry.ExtendedEntry
|
||||
nil, // 45: filer_pb.LookupVolumeResponse.LocationsMapEntry
|
||||
(*LocateBrokerResponse_Resource)(nil), // 46: filer_pb.LocateBrokerResponse.Resource
|
||||
}
|
||||
var file_filer_proto_depIdxs = []int32{
|
||||
4, // 0: filer_pb.LookupDirectoryEntryResponse.entry:type_name -> filer_pb.Entry
|
||||
4, // 1: filer_pb.ListEntriesResponse.entry:type_name -> filer_pb.Entry
|
||||
7, // 2: filer_pb.Entry.chunks:type_name -> filer_pb.FileChunk
|
||||
10, // 3: filer_pb.Entry.attributes:type_name -> filer_pb.FuseAttributes
|
||||
40, // 4: filer_pb.Entry.extended:type_name -> filer_pb.Entry.ExtendedEntry
|
||||
44, // 4: filer_pb.Entry.extended:type_name -> filer_pb.Entry.ExtendedEntry
|
||||
4, // 5: filer_pb.FullEntry.entry:type_name -> filer_pb.Entry
|
||||
4, // 6: filer_pb.EventNotification.old_entry:type_name -> filer_pb.Entry
|
||||
4, // 7: filer_pb.EventNotification.new_entry:type_name -> filer_pb.Entry
|
||||
|
@ -3173,9 +3401,9 @@ var file_filer_proto_depIdxs = []int32{
|
|||
4, // 12: filer_pb.UpdateEntryRequest.entry:type_name -> filer_pb.Entry
|
||||
7, // 13: filer_pb.AppendToEntryRequest.chunks:type_name -> filer_pb.FileChunk
|
||||
25, // 14: filer_pb.Locations.locations:type_name -> filer_pb.Location
|
||||
41, // 15: filer_pb.LookupVolumeResponse.locations_map:type_name -> filer_pb.LookupVolumeResponse.LocationsMapEntry
|
||||
45, // 15: filer_pb.LookupVolumeResponse.locations_map:type_name -> filer_pb.LookupVolumeResponse.LocationsMapEntry
|
||||
6, // 16: filer_pb.SubscribeMetadataResponse.event_notification:type_name -> filer_pb.EventNotification
|
||||
42, // 17: filer_pb.LocateBrokerResponse.resources:type_name -> filer_pb.LocateBrokerResponse.Resource
|
||||
46, // 17: filer_pb.LocateBrokerResponse.resources:type_name -> filer_pb.LocateBrokerResponse.Resource
|
||||
24, // 18: filer_pb.LookupVolumeResponse.LocationsMapEntry.value:type_name -> filer_pb.Locations
|
||||
0, // 19: filer_pb.SeaweedFiler.LookupDirectoryEntry:input_type -> filer_pb.LookupDirectoryEntryRequest
|
||||
2, // 20: filer_pb.SeaweedFiler.ListEntries:input_type -> filer_pb.ListEntriesRequest
|
||||
|
@ -3193,24 +3421,28 @@ var file_filer_proto_depIdxs = []int32{
|
|||
33, // 32: filer_pb.SeaweedFiler.SubscribeLocalMetadata:input_type -> filer_pb.SubscribeMetadataRequest
|
||||
36, // 33: filer_pb.SeaweedFiler.KeepConnected:input_type -> filer_pb.KeepConnectedRequest
|
||||
38, // 34: filer_pb.SeaweedFiler.LocateBroker:input_type -> filer_pb.LocateBrokerRequest
|
||||
1, // 35: filer_pb.SeaweedFiler.LookupDirectoryEntry:output_type -> filer_pb.LookupDirectoryEntryResponse
|
||||
3, // 36: filer_pb.SeaweedFiler.ListEntries:output_type -> filer_pb.ListEntriesResponse
|
||||
12, // 37: filer_pb.SeaweedFiler.CreateEntry:output_type -> filer_pb.CreateEntryResponse
|
||||
14, // 38: filer_pb.SeaweedFiler.UpdateEntry:output_type -> filer_pb.UpdateEntryResponse
|
||||
16, // 39: filer_pb.SeaweedFiler.AppendToEntry:output_type -> filer_pb.AppendToEntryResponse
|
||||
18, // 40: filer_pb.SeaweedFiler.DeleteEntry:output_type -> filer_pb.DeleteEntryResponse
|
||||
20, // 41: filer_pb.SeaweedFiler.AtomicRenameEntry:output_type -> filer_pb.AtomicRenameEntryResponse
|
||||
22, // 42: filer_pb.SeaweedFiler.AssignVolume:output_type -> filer_pb.AssignVolumeResponse
|
||||
26, // 43: filer_pb.SeaweedFiler.LookupVolume:output_type -> filer_pb.LookupVolumeResponse
|
||||
28, // 44: filer_pb.SeaweedFiler.DeleteCollection:output_type -> filer_pb.DeleteCollectionResponse
|
||||
30, // 45: filer_pb.SeaweedFiler.Statistics:output_type -> filer_pb.StatisticsResponse
|
||||
32, // 46: filer_pb.SeaweedFiler.GetFilerConfiguration:output_type -> filer_pb.GetFilerConfigurationResponse
|
||||
34, // 47: filer_pb.SeaweedFiler.SubscribeMetadata:output_type -> filer_pb.SubscribeMetadataResponse
|
||||
34, // 48: filer_pb.SeaweedFiler.SubscribeLocalMetadata:output_type -> filer_pb.SubscribeMetadataResponse
|
||||
37, // 49: filer_pb.SeaweedFiler.KeepConnected:output_type -> filer_pb.KeepConnectedResponse
|
||||
39, // 50: filer_pb.SeaweedFiler.LocateBroker:output_type -> filer_pb.LocateBrokerResponse
|
||||
35, // [35:51] is the sub-list for method output_type
|
||||
19, // [19:35] is the sub-list for method input_type
|
||||
40, // 35: filer_pb.SeaweedFiler.KvGet:input_type -> filer_pb.KvGetRequest
|
||||
42, // 36: filer_pb.SeaweedFiler.KvPut:input_type -> filer_pb.KvPutRequest
|
||||
1, // 37: filer_pb.SeaweedFiler.LookupDirectoryEntry:output_type -> filer_pb.LookupDirectoryEntryResponse
|
||||
3, // 38: filer_pb.SeaweedFiler.ListEntries:output_type -> filer_pb.ListEntriesResponse
|
||||
12, // 39: filer_pb.SeaweedFiler.CreateEntry:output_type -> filer_pb.CreateEntryResponse
|
||||
14, // 40: filer_pb.SeaweedFiler.UpdateEntry:output_type -> filer_pb.UpdateEntryResponse
|
||||
16, // 41: filer_pb.SeaweedFiler.AppendToEntry:output_type -> filer_pb.AppendToEntryResponse
|
||||
18, // 42: filer_pb.SeaweedFiler.DeleteEntry:output_type -> filer_pb.DeleteEntryResponse
|
||||
20, // 43: filer_pb.SeaweedFiler.AtomicRenameEntry:output_type -> filer_pb.AtomicRenameEntryResponse
|
||||
22, // 44: filer_pb.SeaweedFiler.AssignVolume:output_type -> filer_pb.AssignVolumeResponse
|
||||
26, // 45: filer_pb.SeaweedFiler.LookupVolume:output_type -> filer_pb.LookupVolumeResponse
|
||||
28, // 46: filer_pb.SeaweedFiler.DeleteCollection:output_type -> filer_pb.DeleteCollectionResponse
|
||||
30, // 47: filer_pb.SeaweedFiler.Statistics:output_type -> filer_pb.StatisticsResponse
|
||||
32, // 48: filer_pb.SeaweedFiler.GetFilerConfiguration:output_type -> filer_pb.GetFilerConfigurationResponse
|
||||
34, // 49: filer_pb.SeaweedFiler.SubscribeMetadata:output_type -> filer_pb.SubscribeMetadataResponse
|
||||
34, // 50: filer_pb.SeaweedFiler.SubscribeLocalMetadata:output_type -> filer_pb.SubscribeMetadataResponse
|
||||
37, // 51: filer_pb.SeaweedFiler.KeepConnected:output_type -> filer_pb.KeepConnectedResponse
|
||||
39, // 52: filer_pb.SeaweedFiler.LocateBroker:output_type -> filer_pb.LocateBrokerResponse
|
||||
41, // 53: filer_pb.SeaweedFiler.KvGet:output_type -> filer_pb.KvGetResponse
|
||||
43, // 54: filer_pb.SeaweedFiler.KvPut:output_type -> filer_pb.KvPutResponse
|
||||
37, // [37:55] is the sub-list for method output_type
|
||||
19, // [19:37] is the sub-list for method input_type
|
||||
19, // [19:19] is the sub-list for extension type_name
|
||||
19, // [19:19] is the sub-list for extension extendee
|
||||
0, // [0:19] is the sub-list for field type_name
|
||||
|
@ -3702,7 +3934,55 @@ func file_filer_proto_init() {
|
|||
return nil
|
||||
}
|
||||
}
|
||||
file_filer_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*KvGetRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_filer_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*KvGetResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_filer_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*KvPutRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_filer_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*KvPutResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_filer_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*LocateBrokerResponse_Resource); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
|
@ -3721,7 +4001,7 @@ func file_filer_proto_init() {
|
|||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_filer_proto_rawDesc,
|
||||
NumEnums: 0,
|
||||
NumMessages: 43,
|
||||
NumMessages: 47,
|
||||
NumExtensions: 0,
|
||||
NumServices: 1,
|
||||
},
|
||||
|
@ -3763,6 +4043,8 @@ type SeaweedFilerClient interface {
|
|||
SubscribeLocalMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeLocalMetadataClient, error)
|
||||
KeepConnected(ctx context.Context, opts ...grpc.CallOption) (SeaweedFiler_KeepConnectedClient, error)
|
||||
LocateBroker(ctx context.Context, in *LocateBrokerRequest, opts ...grpc.CallOption) (*LocateBrokerResponse, error)
|
||||
KvGet(ctx context.Context, in *KvGetRequest, opts ...grpc.CallOption) (*KvGetResponse, error)
|
||||
KvPut(ctx context.Context, in *KvPutRequest, opts ...grpc.CallOption) (*KvPutResponse, error)
|
||||
}
|
||||
|
||||
type seaweedFilerClient struct {
|
||||
|
@ -4008,6 +4290,24 @@ func (c *seaweedFilerClient) LocateBroker(ctx context.Context, in *LocateBrokerR
|
|||
return out, nil
|
||||
}
|
||||
|
||||
func (c *seaweedFilerClient) KvGet(ctx context.Context, in *KvGetRequest, opts ...grpc.CallOption) (*KvGetResponse, error) {
|
||||
out := new(KvGetResponse)
|
||||
err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/KvGet", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *seaweedFilerClient) KvPut(ctx context.Context, in *KvPutRequest, opts ...grpc.CallOption) (*KvPutResponse, error) {
|
||||
out := new(KvPutResponse)
|
||||
err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/KvPut", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
// SeaweedFilerServer is the server API for SeaweedFiler service.
|
||||
type SeaweedFilerServer interface {
|
||||
LookupDirectoryEntry(context.Context, *LookupDirectoryEntryRequest) (*LookupDirectoryEntryResponse, error)
|
||||
|
@ -4026,6 +4326,8 @@ type SeaweedFilerServer interface {
|
|||
SubscribeLocalMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeLocalMetadataServer) error
|
||||
KeepConnected(SeaweedFiler_KeepConnectedServer) error
|
||||
LocateBroker(context.Context, *LocateBrokerRequest) (*LocateBrokerResponse, error)
|
||||
KvGet(context.Context, *KvGetRequest) (*KvGetResponse, error)
|
||||
KvPut(context.Context, *KvPutRequest) (*KvPutResponse, error)
|
||||
}
|
||||
|
||||
// UnimplementedSeaweedFilerServer can be embedded to have forward compatible implementations.
|
||||
|
@ -4080,6 +4382,12 @@ func (*UnimplementedSeaweedFilerServer) KeepConnected(SeaweedFiler_KeepConnected
|
|||
func (*UnimplementedSeaweedFilerServer) LocateBroker(context.Context, *LocateBrokerRequest) (*LocateBrokerResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method LocateBroker not implemented")
|
||||
}
|
||||
func (*UnimplementedSeaweedFilerServer) KvGet(context.Context, *KvGetRequest) (*KvGetResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method KvGet not implemented")
|
||||
}
|
||||
func (*UnimplementedSeaweedFilerServer) KvPut(context.Context, *KvPutRequest) (*KvPutResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method KvPut not implemented")
|
||||
}
|
||||
|
||||
func RegisterSeaweedFilerServer(s *grpc.Server, srv SeaweedFilerServer) {
|
||||
s.RegisterService(&_SeaweedFiler_serviceDesc, srv)
|
||||
|
@ -4390,6 +4698,42 @@ func _SeaweedFiler_LocateBroker_Handler(srv interface{}, ctx context.Context, de
|
|||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _SeaweedFiler_KvGet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(KvGetRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(SeaweedFilerServer).KvGet(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/filer_pb.SeaweedFiler/KvGet",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(SeaweedFilerServer).KvGet(ctx, req.(*KvGetRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _SeaweedFiler_KvPut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(KvPutRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(SeaweedFilerServer).KvPut(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/filer_pb.SeaweedFiler/KvPut",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(SeaweedFilerServer).KvPut(ctx, req.(*KvPutRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "filer_pb.SeaweedFiler",
|
||||
HandlerType: (*SeaweedFilerServer)(nil),
|
||||
|
@ -4442,6 +4786,14 @@ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
|
|||
MethodName: "LocateBroker",
|
||||
Handler: _SeaweedFiler_LocateBroker_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "KvGet",
|
||||
Handler: _SeaweedFiler_KvGet_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "KvPut",
|
||||
Handler: _SeaweedFiler_KvPut_Handler,
|
||||
},
|
||||
},
|
||||
Streams: []grpc.StreamDesc{
|
||||
{
|
||||
|
|
|
@ -214,7 +214,7 @@ func MkFile(filerClient FilerClient, parentDirectoryPath string, fileName string
|
|||
})
|
||||
}
|
||||
|
||||
func Remove(filerClient FilerClient, parentDirectoryPath, name string, isDeleteData, isRecursive, ignoreRecursiveErr, isFromOtherCluster bool, signature int32) error {
|
||||
func Remove(filerClient FilerClient, parentDirectoryPath, name string, isDeleteData, isRecursive, ignoreRecursiveErr, isFromOtherCluster bool, signatures []int32) error {
|
||||
return filerClient.WithFilerClient(func(client SeaweedFilerClient) error {
|
||||
|
||||
deleteEntryRequest := &DeleteEntryRequest{
|
||||
|
@ -224,9 +224,7 @@ func Remove(filerClient FilerClient, parentDirectoryPath, name string, isDeleteD
|
|||
IsRecursive: isRecursive,
|
||||
IgnoreRecursiveError: ignoreRecursiveErr,
|
||||
IsFromOtherCluster: isFromOtherCluster,
|
||||
}
|
||||
if signature != 0 {
|
||||
deleteEntryRequest.Signatures = []int32{signature}
|
||||
Signatures: signatures,
|
||||
}
|
||||
if resp, err := client.DeleteEntry(context.Background(), deleteEntryRequest); err != nil {
|
||||
if strings.Contains(err.Error(), ErrNotFound.Error()) {
|
||||
|
|
|
@ -59,6 +59,15 @@ func BeforeEntrySerialization(chunks []*FileChunk) {
|
|||
}
|
||||
}
|
||||
|
||||
func EnsureFid(chunk *FileChunk) {
|
||||
if chunk.Fid != nil {
|
||||
return
|
||||
}
|
||||
if fid, err := ToFileIdObject(chunk.FileId); err == nil {
|
||||
chunk.Fid = fid
|
||||
}
|
||||
}
|
||||
|
||||
func AfterEntryDeserialization(chunks []*FileChunk) {
|
||||
|
||||
for _, chunk := range chunks {
|
||||
|
|
|
@ -3,6 +3,8 @@ package replication
|
|||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||
"google.golang.org/grpc"
|
||||
"strings"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
|
@ -43,28 +45,43 @@ func (r *Replicator) Replicate(ctx context.Context, key string, message *filer_p
|
|||
key = newKey
|
||||
if message.OldEntry != nil && message.NewEntry == nil {
|
||||
glog.V(4).Infof("deleting %v", key)
|
||||
return r.sink.DeleteEntry(key, message.OldEntry.IsDirectory, message.DeleteChunks)
|
||||
return r.sink.DeleteEntry(key, message.OldEntry.IsDirectory, message.DeleteChunks, message.Signatures)
|
||||
}
|
||||
if message.OldEntry == nil && message.NewEntry != nil {
|
||||
glog.V(4).Infof("creating %v", key)
|
||||
return r.sink.CreateEntry(key, message.NewEntry)
|
||||
return r.sink.CreateEntry(key, message.NewEntry, message.Signatures)
|
||||
}
|
||||
if message.OldEntry == nil && message.NewEntry == nil {
|
||||
glog.V(0).Infof("weird message %+v", message)
|
||||
return nil
|
||||
}
|
||||
|
||||
foundExisting, err := r.sink.UpdateEntry(key, message.OldEntry, message.NewParentPath, message.NewEntry, message.DeleteChunks)
|
||||
foundExisting, err := r.sink.UpdateEntry(key, message.OldEntry, message.NewParentPath, message.NewEntry, message.DeleteChunks, message.Signatures)
|
||||
if foundExisting {
|
||||
glog.V(4).Infof("updated %v", key)
|
||||
return err
|
||||
}
|
||||
|
||||
err = r.sink.DeleteEntry(key, message.OldEntry.IsDirectory, false)
|
||||
err = r.sink.DeleteEntry(key, message.OldEntry.IsDirectory, false, message.Signatures)
|
||||
if err != nil {
|
||||
return fmt.Errorf("delete old entry %v: %v", key, err)
|
||||
}
|
||||
|
||||
glog.V(4).Infof("creating missing %v", key)
|
||||
return r.sink.CreateEntry(key, message.NewEntry)
|
||||
return r.sink.CreateEntry(key, message.NewEntry, message.Signatures)
|
||||
}
|
||||
|
||||
func ReadFilerSignature(grpcDialOption grpc.DialOption, filer string) (filerSignature int32, readErr error) {
|
||||
if readErr = pb.WithFilerClient(filer, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
if resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{}); err != nil {
|
||||
return fmt.Errorf("GetFilerConfiguration %s: %v", filer, err)
|
||||
} else {
|
||||
filerSignature = resp.Signature
|
||||
}
|
||||
return nil
|
||||
}); readErr != nil {
|
||||
return 0, readErr
|
||||
}
|
||||
return filerSignature, nil
|
||||
}
|
||||
|
||||
|
|
|
@ -70,7 +70,7 @@ func (g *AzureSink) initialize(accountName, accountKey, container, dir string) e
|
|||
return nil
|
||||
}
|
||||
|
||||
func (g *AzureSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error {
|
||||
func (g *AzureSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool, signatures []int32) error {
|
||||
|
||||
key = cleanKey(key)
|
||||
|
||||
|
@ -87,7 +87,7 @@ func (g *AzureSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks boo
|
|||
|
||||
}
|
||||
|
||||
func (g *AzureSink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
||||
func (g *AzureSink) CreateEntry(key string, entry *filer_pb.Entry, signatures []int32) error {
|
||||
|
||||
key = cleanKey(key)
|
||||
|
||||
|
@ -132,7 +132,7 @@ func (g *AzureSink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
|||
|
||||
}
|
||||
|
||||
func (g *AzureSink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
|
||||
func (g *AzureSink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool, signatures []int32) (foundExistingEntry bool, err error) {
|
||||
key = cleanKey(key)
|
||||
// TODO improve efficiency
|
||||
return false, nil
|
||||
|
|
|
@ -57,7 +57,7 @@ func (g *B2Sink) initialize(accountId, accountKey, bucket, dir string) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (g *B2Sink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error {
|
||||
func (g *B2Sink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool, signatures []int32) error {
|
||||
|
||||
key = cleanKey(key)
|
||||
|
||||
|
@ -76,7 +76,7 @@ func (g *B2Sink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool)
|
|||
|
||||
}
|
||||
|
||||
func (g *B2Sink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
||||
func (g *B2Sink) CreateEntry(key string, entry *filer_pb.Entry, signatures []int32) error {
|
||||
|
||||
key = cleanKey(key)
|
||||
|
||||
|
@ -123,7 +123,7 @@ func (g *B2Sink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
|||
|
||||
}
|
||||
|
||||
func (g *B2Sink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
|
||||
func (g *B2Sink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool, signatures []int32) (foundExistingEntry bool, err error) {
|
||||
|
||||
key = cleanKey(key)
|
||||
|
||||
|
|
|
@ -25,7 +25,6 @@ type FilerSink struct {
|
|||
ttlSec int32
|
||||
dataCenter string
|
||||
grpcDialOption grpc.DialOption
|
||||
signature int32
|
||||
}
|
||||
|
||||
func init() {
|
||||
|
@ -41,37 +40,36 @@ func (fs *FilerSink) GetSinkToDirectory() string {
|
|||
}
|
||||
|
||||
func (fs *FilerSink) Initialize(configuration util.Configuration, prefix string) error {
|
||||
return fs.initialize(
|
||||
return fs.DoInitialize(
|
||||
configuration.GetString(prefix+"grpcAddress"),
|
||||
configuration.GetString(prefix+"directory"),
|
||||
configuration.GetString(prefix+"replication"),
|
||||
configuration.GetString(prefix+"collection"),
|
||||
configuration.GetInt(prefix+"ttlSec"),
|
||||
)
|
||||
security.LoadClientTLS(util.GetViper(), "grpc.client"))
|
||||
}
|
||||
|
||||
func (fs *FilerSink) SetSourceFiler(s *source.FilerSource) {
|
||||
fs.filerSource = s
|
||||
}
|
||||
|
||||
func (fs *FilerSink) initialize(grpcAddress string, dir string,
|
||||
replication string, collection string, ttlSec int) (err error) {
|
||||
func (fs *FilerSink) DoInitialize(grpcAddress string, dir string,
|
||||
replication string, collection string, ttlSec int, grpcDialOption grpc.DialOption) (err error) {
|
||||
fs.grpcAddress = grpcAddress
|
||||
fs.dir = dir
|
||||
fs.replication = replication
|
||||
fs.collection = collection
|
||||
fs.ttlSec = int32(ttlSec)
|
||||
fs.grpcDialOption = security.LoadClientTLS(util.GetViper(), "grpc.client")
|
||||
fs.signature = util.RandomInt32()
|
||||
fs.grpcDialOption = grpcDialOption
|
||||
return nil
|
||||
}
|
||||
|
||||
func (fs *FilerSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error {
|
||||
func (fs *FilerSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool, signatures []int32) error {
|
||||
|
||||
dir, name := util.FullPath(key).DirAndName()
|
||||
|
||||
glog.V(1).Infof("delete entry: %v", key)
|
||||
err := filer_pb.Remove(fs, dir, name, deleteIncludeChunks, false, false, true, fs.signature)
|
||||
glog.V(4).Infof("delete entry: %v", key)
|
||||
err := filer_pb.Remove(fs, dir, name, deleteIncludeChunks, true, true, true, signatures)
|
||||
if err != nil {
|
||||
glog.V(0).Infof("delete entry %s: %v", key, err)
|
||||
return fmt.Errorf("delete entry %s: %v", key, err)
|
||||
|
@ -79,7 +77,7 @@ func (fs *FilerSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bo
|
|||
return nil
|
||||
}
|
||||
|
||||
func (fs *FilerSink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
||||
func (fs *FilerSink) CreateEntry(key string, entry *filer_pb.Entry, signatures []int32) error {
|
||||
|
||||
return fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
|
||||
|
@ -93,7 +91,7 @@ func (fs *FilerSink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
|||
glog.V(1).Infof("lookup: %v", lookupRequest)
|
||||
if resp, err := filer_pb.LookupEntry(client, lookupRequest); err == nil {
|
||||
if filer.ETag(resp.Entry) == filer.ETag(entry) {
|
||||
glog.V(0).Infof("already replicated %s", key)
|
||||
glog.V(3).Infof("already replicated %s", key)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
@ -101,11 +99,11 @@ func (fs *FilerSink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
|||
replicatedChunks, err := fs.replicateChunks(entry.Chunks, dir)
|
||||
|
||||
if err != nil {
|
||||
glog.V(0).Infof("replicate entry chunks %s: %v", key, err)
|
||||
return fmt.Errorf("replicate entry chunks %s: %v", key, err)
|
||||
// only warning here since the source chunk may have been deleted already
|
||||
glog.Warningf("replicate entry chunks %s: %v", key, err)
|
||||
}
|
||||
|
||||
glog.V(0).Infof("replicated %s %+v ===> %+v", key, entry.Chunks, replicatedChunks)
|
||||
glog.V(4).Infof("replicated %s %+v ===> %+v", key, entry.Chunks, replicatedChunks)
|
||||
|
||||
request := &filer_pb.CreateEntryRequest{
|
||||
Directory: dir,
|
||||
|
@ -116,10 +114,10 @@ func (fs *FilerSink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
|||
Chunks: replicatedChunks,
|
||||
},
|
||||
IsFromOtherCluster: true,
|
||||
Signatures: []int32{fs.signature},
|
||||
Signatures: signatures,
|
||||
}
|
||||
|
||||
glog.V(1).Infof("create: %v", request)
|
||||
glog.V(3).Infof("create: %v", request)
|
||||
if err := filer_pb.CreateEntry(client, request); err != nil {
|
||||
glog.V(0).Infof("create entry %s: %v", key, err)
|
||||
return fmt.Errorf("create entry %s: %v", key, err)
|
||||
|
@ -129,7 +127,7 @@ func (fs *FilerSink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
|||
})
|
||||
}
|
||||
|
||||
func (fs *FilerSink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
|
||||
func (fs *FilerSink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool, signatures []int32) (foundExistingEntry bool, err error) {
|
||||
|
||||
dir, name := util.FullPath(key).DirAndName()
|
||||
|
||||
|
@ -158,16 +156,16 @@ func (fs *FilerSink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParent
|
|||
return false, fmt.Errorf("lookup %s: %v", key, err)
|
||||
}
|
||||
|
||||
glog.V(0).Infof("oldEntry %+v, newEntry %+v, existingEntry: %+v", oldEntry, newEntry, existingEntry)
|
||||
glog.V(4).Infof("oldEntry %+v, newEntry %+v, existingEntry: %+v", oldEntry, newEntry, existingEntry)
|
||||
|
||||
if existingEntry.Attributes.Mtime > newEntry.Attributes.Mtime {
|
||||
// skip if already changed
|
||||
// this usually happens when the messages are not ordered
|
||||
glog.V(0).Infof("late updates %s", key)
|
||||
glog.V(2).Infof("late updates %s", key)
|
||||
} else if filer.ETag(newEntry) == filer.ETag(existingEntry) {
|
||||
// skip if no change
|
||||
// this usually happens when retrying the replication
|
||||
glog.V(0).Infof("already replicated %s", key)
|
||||
glog.V(3).Infof("already replicated %s", key)
|
||||
} else {
|
||||
// find out what changed
|
||||
deletedChunks, newChunks, err := compareChunks(filer.LookupFn(fs), oldEntry, newEntry)
|
||||
|
@ -196,7 +194,7 @@ func (fs *FilerSink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParent
|
|||
Directory: newParentPath,
|
||||
Entry: existingEntry,
|
||||
IsFromOtherCluster: true,
|
||||
Signatures: []int32{fs.signature},
|
||||
Signatures: signatures,
|
||||
}
|
||||
|
||||
if _, err := client.UpdateEntry(context.Background(), request); err != nil {
|
||||
|
|
|
@ -69,7 +69,7 @@ func (g *GcsSink) initialize(google_application_credentials, bucketName, dir str
|
|||
return nil
|
||||
}
|
||||
|
||||
func (g *GcsSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error {
|
||||
func (g *GcsSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool, signatures []int32) error {
|
||||
|
||||
if isDirectory {
|
||||
key = key + "/"
|
||||
|
@ -83,7 +83,7 @@ func (g *GcsSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool)
|
|||
|
||||
}
|
||||
|
||||
func (g *GcsSink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
||||
func (g *GcsSink) CreateEntry(key string, entry *filer_pb.Entry, signatures []int32) error {
|
||||
|
||||
if entry.IsDirectory {
|
||||
return nil
|
||||
|
@ -119,7 +119,7 @@ func (g *GcsSink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
|||
|
||||
}
|
||||
|
||||
func (g *GcsSink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
|
||||
func (g *GcsSink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool, signatures []int32) (foundExistingEntry bool, err error) {
|
||||
// TODO improve efficiency
|
||||
return false, nil
|
||||
}
|
||||
|
|
|
@ -9,9 +9,9 @@ import (
|
|||
type ReplicationSink interface {
|
||||
GetName() string
|
||||
Initialize(configuration util.Configuration, prefix string) error
|
||||
DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error
|
||||
CreateEntry(key string, entry *filer_pb.Entry) error
|
||||
UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error)
|
||||
DeleteEntry(key string, isDirectory, deleteIncludeChunks bool, signatures []int32) error
|
||||
CreateEntry(key string, entry *filer_pb.Entry, signatures []int32) error
|
||||
UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool, signatures []int32) (foundExistingEntry bool, err error)
|
||||
GetSinkToDirectory() string
|
||||
SetSourceFiler(s *source.FilerSource)
|
||||
}
|
||||
|
|
|
@ -83,7 +83,7 @@ func (s3sink *S3Sink) initialize(awsAccessKeyId, awsSecretAccessKey, region, buc
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s3sink *S3Sink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error {
|
||||
func (s3sink *S3Sink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool, signatures []int32) error {
|
||||
|
||||
key = cleanKey(key)
|
||||
|
||||
|
@ -95,7 +95,7 @@ func (s3sink *S3Sink) DeleteEntry(key string, isDirectory, deleteIncludeChunks b
|
|||
|
||||
}
|
||||
|
||||
func (s3sink *S3Sink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
||||
func (s3sink *S3Sink) CreateEntry(key string, entry *filer_pb.Entry, signatures []int32) error {
|
||||
key = cleanKey(key)
|
||||
|
||||
if entry.IsDirectory {
|
||||
|
@ -136,7 +136,7 @@ func (s3sink *S3Sink) CreateEntry(key string, entry *filer_pb.Entry) error {
|
|||
|
||||
}
|
||||
|
||||
func (s3sink *S3Sink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
|
||||
func (s3sink *S3Sink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool, signatures []int32) (foundExistingEntry bool, err error) {
|
||||
key = cleanKey(key)
|
||||
// TODO improve efficiency
|
||||
return false, nil
|
||||
|
|
|
@ -28,13 +28,13 @@ type FilerSource struct {
|
|||
}
|
||||
|
||||
func (fs *FilerSource) Initialize(configuration util.Configuration, prefix string) error {
|
||||
return fs.initialize(
|
||||
return fs.DoInitialize(
|
||||
configuration.GetString(prefix+"grpcAddress"),
|
||||
configuration.GetString(prefix+"directory"),
|
||||
)
|
||||
}
|
||||
|
||||
func (fs *FilerSource) initialize(grpcAddress string, dir string) (err error) {
|
||||
func (fs *FilerSource) DoInitialize(grpcAddress string, dir string) (err error) {
|
||||
fs.grpcAddress = grpcAddress
|
||||
fs.Dir = dir
|
||||
fs.grpcDialOption = security.LoadClientTLS(util.GetViper(), "grpc.client")
|
||||
|
|
|
@ -314,7 +314,7 @@ func (fs *FilerServer) DeleteEntry(ctx context.Context, req *filer_pb.DeleteEntr
|
|||
|
||||
glog.V(4).Infof("DeleteEntry %v", req)
|
||||
|
||||
err = fs.filer.DeleteEntryMetaAndData(ctx, util.JoinPath(req.Directory, req.Name), req.IsRecursive, req.IgnoreRecursiveError, req.IsDeleteData, req.IsFromOtherCluster, nil)
|
||||
err = fs.filer.DeleteEntryMetaAndData(ctx, util.JoinPath(req.Directory, req.Name), req.IsRecursive, req.IgnoreRecursiveError, req.IsDeleteData, req.IsFromOtherCluster, req.Signatures)
|
||||
resp = &filer_pb.DeleteEntryResponse{}
|
||||
if err != nil {
|
||||
resp.Error = err.Error()
|
||||
|
|
42
weed/server/filer_grpc_server_kv.go
Normal file
42
weed/server/filer_grpc_server_kv.go
Normal file
|
@ -0,0 +1,42 @@
|
|||
package weed_server
|
||||
|
||||
import (
|
||||
"context"
|
||||
"github.com/chrislusf/seaweedfs/weed/filer"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
)
|
||||
|
||||
func (fs *FilerServer) KvGet(ctx context.Context, req *filer_pb.KvGetRequest) (*filer_pb.KvGetResponse, error) {
|
||||
|
||||
value, err := fs.filer.Store.KvGet(ctx, req.Key)
|
||||
if err == filer.ErrKvNotFound {
|
||||
return &filer_pb.KvGetResponse{}, nil
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
return &filer_pb.KvGetResponse{Error: err.Error()}, nil
|
||||
}
|
||||
|
||||
return &filer_pb.KvGetResponse{
|
||||
Value: value,
|
||||
}, nil
|
||||
|
||||
}
|
||||
|
||||
// KvPut sets the key~value. if empty value, delete the kv entry
|
||||
func (fs *FilerServer) KvPut(ctx context.Context, req *filer_pb.KvPutRequest) (*filer_pb.KvPutResponse, error) {
|
||||
|
||||
if len(req.Value) == 0 {
|
||||
if err := fs.filer.Store.KvDelete(ctx, req.Key); err != nil {
|
||||
return &filer_pb.KvPutResponse{Error: err.Error()}, nil
|
||||
}
|
||||
}
|
||||
|
||||
err := fs.filer.Store.KvPut(ctx, req.Key, req.Value)
|
||||
if err != nil {
|
||||
return &filer_pb.KvPutResponse{Error: err.Error()}, nil
|
||||
}
|
||||
|
||||
return &filer_pb.KvPutResponse{}, nil
|
||||
|
||||
}
|
|
@ -2,6 +2,7 @@ package weed_server
|
|||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/util/log_buffer"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
|
@ -24,7 +25,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
|
|||
lastReadTime := time.Unix(0, req.SinceNs)
|
||||
glog.V(0).Infof(" %v starts to subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
|
||||
|
||||
eachEventNotificationFn := eachEventNotificationFn(req, stream, clientName, req.Signature)
|
||||
eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName, req.Signature)
|
||||
|
||||
eachLogEntryFn := eachLogEntryFn(eachEventNotificationFn)
|
||||
|
||||
|
@ -37,12 +38,21 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
|
|||
lastReadTime = time.Unix(0, processedTsNs)
|
||||
}
|
||||
|
||||
err = fs.filer.MetaAggregator.MetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool {
|
||||
for {
|
||||
lastReadTime, err = fs.filer.MetaAggregator.MetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool {
|
||||
fs.filer.MetaAggregator.ListenersLock.Lock()
|
||||
fs.filer.MetaAggregator.ListenersCond.Wait()
|
||||
fs.filer.MetaAggregator.ListenersLock.Unlock()
|
||||
return true
|
||||
}, eachLogEntryFn)
|
||||
if err != nil {
|
||||
glog.Errorf("processed to %v: %v", lastReadTime, err)
|
||||
time.Sleep(3127 * time.Millisecond)
|
||||
if err != log_buffer.ResumeError {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return err
|
||||
|
||||
|
@ -59,7 +69,7 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
|
|||
lastReadTime := time.Unix(0, req.SinceNs)
|
||||
glog.V(0).Infof(" %v local subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
|
||||
|
||||
eachEventNotificationFn := eachEventNotificationFn(req, stream, clientName, req.Signature)
|
||||
eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName, req.Signature)
|
||||
|
||||
eachLogEntryFn := eachLogEntryFn(eachEventNotificationFn)
|
||||
|
||||
|
@ -75,12 +85,21 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
|
|||
glog.V(0).Infof("after local log reads, %v local subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
|
||||
|
||||
// println("reading from in memory logs ...")
|
||||
err = fs.filer.LocalMetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool {
|
||||
for {
|
||||
lastReadTime, err = fs.filer.LocalMetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool {
|
||||
fs.listenersLock.Lock()
|
||||
fs.listenersCond.Wait()
|
||||
fs.listenersLock.Unlock()
|
||||
return true
|
||||
}, eachLogEntryFn)
|
||||
if err != nil {
|
||||
glog.Errorf("processed to %v: %v", lastReadTime, err)
|
||||
time.Sleep(3127 * time.Millisecond)
|
||||
if err != log_buffer.ResumeError {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return err
|
||||
|
||||
|
@ -102,13 +121,20 @@ func eachLogEntryFn(eachEventNotificationFn func(dirPath string, eventNotificati
|
|||
}
|
||||
}
|
||||
|
||||
func eachEventNotificationFn(req *filer_pb.SubscribeMetadataRequest, stream filer_pb.SeaweedFiler_SubscribeMetadataServer, clientName string, clientSignature int32) func(dirPath string, eventNotification *filer_pb.EventNotification, tsNs int64) error {
|
||||
func (fs *FilerServer) eachEventNotificationFn(req *filer_pb.SubscribeMetadataRequest, stream filer_pb.SeaweedFiler_SubscribeMetadataServer, clientName string, clientSignature int32) func(dirPath string, eventNotification *filer_pb.EventNotification, tsNs int64) error {
|
||||
return func(dirPath string, eventNotification *filer_pb.EventNotification, tsNs int64) error {
|
||||
|
||||
foundSelf := false
|
||||
for _, sig := range eventNotification.Signatures {
|
||||
if sig == clientSignature && clientSignature != 0 {
|
||||
return nil
|
||||
}
|
||||
if sig == fs.filer.Signature {
|
||||
foundSelf = true
|
||||
}
|
||||
}
|
||||
if !foundSelf {
|
||||
eventNotification.Signatures = append(eventNotification.Signatures, fs.filer.Signature)
|
||||
}
|
||||
|
||||
// get complete path to the file or directory
|
||||
|
|
|
@ -259,7 +259,7 @@ func (fs *WebDavFileSystem) removeAll(ctx context.Context, fullFilePath string)
|
|||
|
||||
dir, name := util.FullPath(fullFilePath).DirAndName()
|
||||
|
||||
return filer_pb.Remove(fs, dir, name, true, false, false, false, fs.signature)
|
||||
return filer_pb.Remove(fs, dir, name, true, false, false, false, []int32{fs.signature})
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -49,6 +49,6 @@ func (c *commandBucketDelete) Do(args []string, commandEnv *CommandEnv, writer i
|
|||
return fmt.Errorf("read buckets: %v", err)
|
||||
}
|
||||
|
||||
return filer_pb.Remove(commandEnv, filerBucketsPath, *bucketName, false, true, true, false, 0)
|
||||
return filer_pb.Remove(commandEnv, filerBucketsPath, *bucketName, false, true, true, false, nil)
|
||||
|
||||
}
|
||||
|
|
|
@ -79,6 +79,10 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv,
|
|||
allLocations = append(allLocations, loc)
|
||||
})
|
||||
|
||||
if len(allLocations) == 0 {
|
||||
return fmt.Errorf("no data nodes at all")
|
||||
}
|
||||
|
||||
// find all under replicated volumes
|
||||
var underReplicatedVolumeIds, overReplicatedVolumeIds []uint32
|
||||
for vid, replicas := range volumeReplicas {
|
||||
|
@ -100,10 +104,6 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv,
|
|||
return nil
|
||||
}
|
||||
|
||||
if len(allLocations) == 0 {
|
||||
return fmt.Errorf("no data nodes at all")
|
||||
}
|
||||
|
||||
// find the most under populated data nodes
|
||||
keepDataNodesSorted(allLocations)
|
||||
|
||||
|
|
|
@ -2,6 +2,7 @@ package log_buffer
|
|||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
@ -11,13 +12,17 @@ import (
|
|||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
)
|
||||
|
||||
var (
|
||||
ResumeError = fmt.Errorf("resume")
|
||||
)
|
||||
|
||||
func (logBuffer *LogBuffer) LoopProcessLogData(
|
||||
startTreadTime time.Time,
|
||||
waitForDataFn func() bool,
|
||||
eachLogDataFn func(logEntry *filer_pb.LogEntry) error) (err error) {
|
||||
eachLogDataFn func(logEntry *filer_pb.LogEntry) error) (lastReadTime time.Time, err error) {
|
||||
// loop through all messages
|
||||
var bytesBuf *bytes.Buffer
|
||||
lastReadTime := startTreadTime
|
||||
lastReadTime = startTreadTime
|
||||
defer func() {
|
||||
if bytesBuf != nil {
|
||||
logBuffer.ReleaseMemory(bytesBuf)
|
||||
|
@ -48,10 +53,13 @@ func (logBuffer *LogBuffer) LoopProcessLogData(
|
|||
for pos := 0; pos+4 < len(buf); {
|
||||
|
||||
size := util.BytesToUint32(buf[pos : pos+4])
|
||||
if pos+4+int(size) > len(buf) {
|
||||
err = ResumeError
|
||||
glog.Errorf("LoopProcessLogData: read buffer %v read %d [%d,%d) from [0,%d)", lastReadTime, batchSize, pos, pos+int(size)+4, len(buf))
|
||||
return
|
||||
}
|
||||
entryData := buf[pos+4 : pos+4+int(size)]
|
||||
|
||||
// fmt.Printf("read buffer read %d [%d,%d) from [0,%d)\n", batchSize, pos, pos+int(size)+4, len(buf))
|
||||
|
||||
logEntry := &filer_pb.LogEntry{}
|
||||
if err = proto.Unmarshal(entryData, logEntry); err != nil {
|
||||
glog.Errorf("unexpected unmarshal messaging_pb.Message: %v", err)
|
||||
|
|
Loading…
Reference in a new issue