add filer notification

This commit is contained in:
Chris Lu 2018-08-13 01:20:49 -07:00
parent 75d63db60d
commit f036ef8a3c
13 changed files with 428 additions and 101 deletions

View file

@ -32,6 +32,7 @@ type FilerOptions struct {
secretKey *string secretKey *string
dirListingLimit *int dirListingLimit *int
dataCenter *string dataCenter *string
enableNotification *bool
} }
func init() { func init() {
@ -49,6 +50,7 @@ func init() {
f.secretKey = cmdFiler.Flag.String("secure.secret", "", "secret to encrypt Json Web Token(JWT)") f.secretKey = cmdFiler.Flag.String("secure.secret", "", "secret to encrypt Json Web Token(JWT)")
f.dirListingLimit = cmdFiler.Flag.Int("dirListLimit", 1000, "limit sub dir listing size") f.dirListingLimit = cmdFiler.Flag.Int("dirListLimit", 1000, "limit sub dir listing size")
f.dataCenter = cmdFiler.Flag.String("dataCenter", "", "prefer to write to volumes in this data center") f.dataCenter = cmdFiler.Flag.String("dataCenter", "", "prefer to write to volumes in this data center")
f.enableNotification = cmdFiler.Flag.Bool("notify", false, "send file updates to the queue defined in message_queue.toml")
} }
var cmdFiler = &Command{ var cmdFiler = &Command{
@ -90,14 +92,15 @@ func (fo *FilerOptions) start() {
fs, nfs_err := weed_server.NewFilerServer(defaultMux, publicVolumeMux, &weed_server.FilerOption{ fs, nfs_err := weed_server.NewFilerServer(defaultMux, publicVolumeMux, &weed_server.FilerOption{
Masters: strings.Split(*f.masters, ","), Masters: strings.Split(*f.masters, ","),
Collection: *f.collection, Collection: *fo.collection,
DefaultReplication: *f.defaultReplicaPlacement, DefaultReplication: *fo.defaultReplicaPlacement,
RedirectOnRead: *f.redirectOnRead, RedirectOnRead: *fo.redirectOnRead,
DisableDirListing: *f.disableDirListing, DisableDirListing: *fo.disableDirListing,
MaxMB: *f.maxMB, MaxMB: *fo.maxMB,
SecretKey: *f.secretKey, SecretKey: *fo.secretKey,
DirListingLimit: *f.dirListingLimit, DirListingLimit: *fo.dirListingLimit,
DataCenter: *f.dataCenter, DataCenter: *fo.dataCenter,
EnableNotification: *fo.enableNotification,
}) })
if nfs_err != nil { if nfs_err != nil {
glog.Fatalf("Filer startup error: %v", nfs_err) glog.Fatalf("Filer startup error: %v", nfs_err)
@ -127,9 +130,9 @@ func (fo *FilerOptions) start() {
} }
// starting grpc server // starting grpc server
grpcPort := *f.grpcPort grpcPort := *fo.grpcPort
if grpcPort == 0 { if grpcPort == 0 {
grpcPort = *f.port + 10000 grpcPort = *fo.port + 10000
} }
grpcL, err := util.NewListener(":"+strconv.Itoa(grpcPort), 0) grpcL, err := util.NewListener(":"+strconv.Itoa(grpcPort), 0)
if err != nil { if err != nil {

View file

@ -84,12 +84,14 @@ func init() {
serverOptions.cpuprofile = cmdServer.Flag.String("cpuprofile", "", "cpu profile output file") serverOptions.cpuprofile = cmdServer.Flag.String("cpuprofile", "", "cpu profile output file")
filerOptions.collection = cmdServer.Flag.String("filer.collection", "", "all data will be stored in this collection") filerOptions.collection = cmdServer.Flag.String("filer.collection", "", "all data will be stored in this collection")
filerOptions.port = cmdServer.Flag.Int("filer.port", 8888, "filer server http listen port") filerOptions.port = cmdServer.Flag.Int("filer.port", 8888, "filer server http listen port")
filerOptions.grpcPort = cmdServer.Flag.Int("filer.port.grpc", 0, "filer grpc server listen port, default to http port + 10000")
filerOptions.publicPort = cmdServer.Flag.Int("filer.port.public", 0, "filer server public http listen port") filerOptions.publicPort = cmdServer.Flag.Int("filer.port.public", 0, "filer server public http listen port")
filerOptions.defaultReplicaPlacement = cmdServer.Flag.String("filer.defaultReplicaPlacement", "", "Default replication type if not specified during runtime.") filerOptions.defaultReplicaPlacement = cmdServer.Flag.String("filer.defaultReplicaPlacement", "", "Default replication type if not specified during runtime.")
filerOptions.redirectOnRead = cmdServer.Flag.Bool("filer.redirectOnRead", false, "whether proxy or redirect to volume server during file GET request") filerOptions.redirectOnRead = cmdServer.Flag.Bool("filer.redirectOnRead", false, "whether proxy or redirect to volume server during file GET request")
filerOptions.disableDirListing = cmdServer.Flag.Bool("filer.disableDirListing", false, "turn off directory listing") filerOptions.disableDirListing = cmdServer.Flag.Bool("filer.disableDirListing", false, "turn off directory listing")
filerOptions.maxMB = cmdServer.Flag.Int("filer.maxMB", 32, "split files larger than the limit") filerOptions.maxMB = cmdServer.Flag.Int("filer.maxMB", 32, "split files larger than the limit")
filerOptions.dirListingLimit = cmdServer.Flag.Int("filer.dirListLimit", 1000, "limit sub dir listing size") filerOptions.dirListingLimit = cmdServer.Flag.Int("filer.dirListLimit", 1000, "limit sub dir listing size")
filerOptions.enableNotification = cmdServer.Flag.Bool("filer.notify", false, "send file updates to the queue defined in message_queue.toml")
} }
func runServer(cmd *Command, args []string) bool { func runServer(cmd *Command, args []string) bool {

View file

@ -63,3 +63,25 @@ func PbToEntryAttribute(attr *filer_pb.FuseAttributes) Attr {
return t return t
} }
func EqualEntry(a, b *Entry) bool {
if a == b {
return true
}
if a == nil && b != nil || a != nil && b == nil {
return false
}
if !proto.Equal(EntryAttributeToPb(a), EntryAttributeToPb(b)) {
return false
}
if len(a.Chunks) != len(b.Chunks) {
return false
}
for i := 0; i < len(a.Chunks); i++ {
if !proto.Equal(a.Chunks[i], b.Chunks[i]) {
return false
}
}
return true
}

View file

@ -90,6 +90,9 @@ func (f *Filer) CreateEntry(entry *Entry) error {
if mkdirErr != nil { if mkdirErr != nil {
return fmt.Errorf("mkdir %s: %v", dirPath, mkdirErr) return fmt.Errorf("mkdir %s: %v", dirPath, mkdirErr)
} }
f.NotifyUpdateEvent(nil, dirEntry)
} else if !dirEntry.IsDirectory() { } else if !dirEntry.IsDirectory() {
return fmt.Errorf("%s is a file", dirPath) return fmt.Errorf("%s is a file", dirPath)
} }
@ -122,6 +125,8 @@ func (f *Filer) CreateEntry(entry *Entry) error {
return fmt.Errorf("insert entry %s: %v", entry.FullPath, err) return fmt.Errorf("insert entry %s: %v", entry.FullPath, err)
} }
f.NotifyUpdateEvent(oldEntry, entry)
f.deleteChunksIfNotNew(oldEntry, entry) f.deleteChunksIfNotNew(oldEntry, entry)
return nil return nil
@ -170,6 +175,9 @@ func (f *Filer) DeleteEntryMetaAndData(p FullPath, isRecursive bool, shouldDelet
return nil return nil
} }
glog.V(0).Infof("deleting entry %v", p) glog.V(0).Infof("deleting entry %v", p)
f.NotifyUpdateEvent(entry, nil)
return f.store.DeleteEntry(p) return f.store.DeleteEntry(p)
} }

View file

@ -0,0 +1,38 @@
package filer2
import (
"github.com/chrislusf/seaweedfs/weed/msgqueue"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
)
func (f *Filer) NotifyUpdateEvent(oldEntry, newEntry *Entry) {
var key string
if oldEntry != nil {
key = string(oldEntry.FullPath)
} else if newEntry != nil {
key = string(newEntry.FullPath)
} else {
return
}
msgqueue.Queue.SendMessage(
key,
&filer_pb.EventNotification{
OldEntry: toProtoEntry(oldEntry),
NewEntry: toProtoEntry(newEntry),
},
)
}
func toProtoEntry(entry *Entry) *filer_pb.Entry {
if entry == nil {
return nil
}
return &filer_pb.Entry{
Name: string(entry.FullPath),
IsDirectory: entry.IsDirectory(),
Attributes: EntryAttributeToPb(entry),
Chunks: entry.Chunks,
}
}

View file

@ -0,0 +1,82 @@
package msgqueue
import (
"os"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/spf13/viper"
)
const (
MSG_QUEUE_TOML_EXAMPLE = `
# A sample TOML config file for SeaweedFS message queue
[log]
enabled = true
[kafka]
enabled = false
hosts = [
"localhost:9092"
]
topic = "seaweedfs_filer"
`
)
var (
MessageQueues []MessageQueue
Queue MessageQueue
)
func LoadConfiguration() {
// find a filer store
viper.SetConfigName("message_queue") // name of config file (without extension)
viper.AddConfigPath(".") // optionally look for config in the working directory
viper.AddConfigPath("$HOME/.seaweedfs") // call multiple times to add many search paths
viper.AddConfigPath("/etc/seaweedfs/") // path to look for the config file in
if err := viper.ReadInConfig(); err != nil { // Handle errors reading the config file
glog.Fatalf("Failed to load message_queue.toml file from current directory, or $HOME/.seaweedfs/, "+
"or /etc/seaweedfs/"+
"\n\nPlease follow this example and add a message_queue.toml file to "+
"current directory, or $HOME/.seaweedfs/, or /etc/seaweedfs/:\n"+MSG_QUEUE_TOML_EXAMPLE, err)
}
glog.V(0).Infof("Reading message queue configuration from %s", viper.ConfigFileUsed())
for _, store := range MessageQueues {
if viper.GetBool(store.GetName() + ".enabled") {
viperSub := viper.Sub(store.GetName())
if err := store.Initialize(viperSub); err != nil {
glog.Fatalf("Failed to initialize store for %s: %+v",
store.GetName(), err)
}
Queue = store
glog.V(0).Infof("Configure message queue for %s from %s", store.GetName(), viper.ConfigFileUsed())
return
}
}
println()
println("Supported message queues are:")
for _, store := range MessageQueues {
println(" " + store.GetName())
}
println()
println("Please configure a supported message queue in", viper.ConfigFileUsed())
println()
os.Exit(-1)
}
// A simplified interface to decouple from Viper
type Configuration interface {
GetString(key string) string
GetBool(key string) bool
GetInt(key string) int
GetInt64(key string) int64
GetFloat64(key string) float64
GetStringSlice(key string) []string
}

View file

@ -0,0 +1,76 @@
package kafka
import (
_ "github.com/go-sql-driver/mysql"
"github.com/chrislusf/seaweedfs/weed/msgqueue"
"github.com/golang/protobuf/proto"
"github.com/Shopify/sarama"
"github.com/chrislusf/seaweedfs/weed/glog"
)
func init() {
msgqueue.MessageQueues = append(msgqueue.MessageQueues, &KafkaQueue{})
}
type KafkaQueue struct {
topic string
producer sarama.AsyncProducer
}
func (k *KafkaQueue) GetName() string {
return "kafka"
}
func (k *KafkaQueue) Initialize(configuration msgqueue.Configuration) (err error) {
return k.initialize(
configuration.GetStringSlice("hosts"),
configuration.GetString("topic"),
)
}
func (k *KafkaQueue) initialize(hosts []string, topic string) (err error) {
config := sarama.NewConfig()
config.Producer.RequiredAcks = sarama.WaitForLocal
config.Producer.Partitioner = sarama.NewHashPartitioner
config.Producer.Return.Successes = true
config.Producer.Return.Errors = true
k.producer, err = sarama.NewAsyncProducer(hosts, config)
go k.handleSuccess()
go k.handleError()
return nil
}
func (k *KafkaQueue) SendMessage(key string, message proto.Message) (err error) {
bytes, err := proto.Marshal(message)
if err != nil {
return
}
msg := &sarama.ProducerMessage{
Topic: k.topic,
Key: sarama.StringEncoder(key),
Value: sarama.ByteEncoder(bytes),
}
k.producer.Input() <- msg
return nil
}
func (k *KafkaQueue) handleSuccess() {
for {
pm := <-k.producer.Successes()
if pm != nil {
glog.Infof("producer message success, partition:%d offset:%d key:%v valus:%s", pm.Partition, pm.Offset, pm.Key, pm.Value)
}
}
}
func (k *KafkaQueue) handleError() {
for {
err := <-k.producer.Errors()
if err != nil {
glog.Errorf("producer message error, partition:%d offset:%d key:%v valus:%s error(%v)", err.Msg.Partition, err.Msg.Offset, err.Msg.Key, err.Msg.Value, err.Err)
}
}
}

View file

@ -0,0 +1,29 @@
package kafka
import (
_ "github.com/go-sql-driver/mysql"
"github.com/chrislusf/seaweedfs/weed/msgqueue"
"github.com/golang/protobuf/proto"
"github.com/chrislusf/seaweedfs/weed/glog"
)
func init() {
msgqueue.MessageQueues = append(msgqueue.MessageQueues, &LogQueue{})
}
type LogQueue struct {
}
func (k *LogQueue) GetName() string {
return "log"
}
func (k *LogQueue) Initialize(configuration msgqueue.Configuration) (err error) {
return nil
}
func (k *LogQueue) SendMessage(key string, message proto.Message) (err error) {
glog.V(0).Infof("%v: %+v", key, message)
return nil
}

View file

@ -0,0 +1,11 @@
package msgqueue
import "github.com/golang/protobuf/proto"
type MessageQueue interface {
// GetName gets the name to locate the configuration in message_queue.toml file
GetName() string
// Initialize initializes the file store
Initialize(configuration Configuration) error
SendMessage(key string, message proto.Message) error
}

View file

@ -65,6 +65,11 @@ message Entry {
FuseAttributes attributes = 4; FuseAttributes attributes = 4;
} }
message EventNotification {
Entry old_entry = 2;
Entry new_entry = 3;
}
message FileChunk { message FileChunk {
string file_id = 1; string file_id = 1;
int64 offset = 2; int64 offset = 2;

View file

@ -14,6 +14,7 @@ It has these top-level messages:
ListEntriesRequest ListEntriesRequest
ListEntriesResponse ListEntriesResponse
Entry Entry
EventNotification
FileChunk FileChunk
FuseAttributes FuseAttributes
GetEntryAttributesRequest GetEntryAttributesRequest
@ -201,6 +202,38 @@ func (m *Entry) GetAttributes() *FuseAttributes {
return nil return nil
} }
type EventNotification struct {
Name string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
OldEntry *Entry `protobuf:"bytes,2,opt,name=old_entry,json=oldEntry" json:"old_entry,omitempty"`
NewEntry *Entry `protobuf:"bytes,3,opt,name=new_entry,json=newEntry" json:"new_entry,omitempty"`
}
func (m *EventNotification) Reset() { *m = EventNotification{} }
func (m *EventNotification) String() string { return proto.CompactTextString(m) }
func (*EventNotification) ProtoMessage() {}
func (*EventNotification) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} }
func (m *EventNotification) GetName() string {
if m != nil {
return m.Name
}
return ""
}
func (m *EventNotification) GetOldEntry() *Entry {
if m != nil {
return m.OldEntry
}
return nil
}
func (m *EventNotification) GetNewEntry() *Entry {
if m != nil {
return m.NewEntry
}
return nil
}
type FileChunk struct { type FileChunk struct {
FileId string `protobuf:"bytes,1,opt,name=file_id,json=fileId" json:"file_id,omitempty"` FileId string `protobuf:"bytes,1,opt,name=file_id,json=fileId" json:"file_id,omitempty"`
Offset int64 `protobuf:"varint,2,opt,name=offset" json:"offset,omitempty"` Offset int64 `protobuf:"varint,2,opt,name=offset" json:"offset,omitempty"`
@ -211,7 +244,7 @@ type FileChunk struct {
func (m *FileChunk) Reset() { *m = FileChunk{} } func (m *FileChunk) Reset() { *m = FileChunk{} }
func (m *FileChunk) String() string { return proto.CompactTextString(m) } func (m *FileChunk) String() string { return proto.CompactTextString(m) }
func (*FileChunk) ProtoMessage() {} func (*FileChunk) ProtoMessage() {}
func (*FileChunk) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } func (*FileChunk) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} }
func (m *FileChunk) GetFileId() string { func (m *FileChunk) GetFileId() string {
if m != nil { if m != nil {
@ -257,7 +290,7 @@ type FuseAttributes struct {
func (m *FuseAttributes) Reset() { *m = FuseAttributes{} } func (m *FuseAttributes) Reset() { *m = FuseAttributes{} }
func (m *FuseAttributes) String() string { return proto.CompactTextString(m) } func (m *FuseAttributes) String() string { return proto.CompactTextString(m) }
func (*FuseAttributes) ProtoMessage() {} func (*FuseAttributes) ProtoMessage() {}
func (*FuseAttributes) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } func (*FuseAttributes) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} }
func (m *FuseAttributes) GetFileSize() uint64 { func (m *FuseAttributes) GetFileSize() uint64 {
if m != nil { if m != nil {
@ -338,7 +371,7 @@ type GetEntryAttributesRequest struct {
func (m *GetEntryAttributesRequest) Reset() { *m = GetEntryAttributesRequest{} } func (m *GetEntryAttributesRequest) Reset() { *m = GetEntryAttributesRequest{} }
func (m *GetEntryAttributesRequest) String() string { return proto.CompactTextString(m) } func (m *GetEntryAttributesRequest) String() string { return proto.CompactTextString(m) }
func (*GetEntryAttributesRequest) ProtoMessage() {} func (*GetEntryAttributesRequest) ProtoMessage() {}
func (*GetEntryAttributesRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } func (*GetEntryAttributesRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} }
func (m *GetEntryAttributesRequest) GetName() string { func (m *GetEntryAttributesRequest) GetName() string {
if m != nil { if m != nil {
@ -369,7 +402,7 @@ type GetEntryAttributesResponse struct {
func (m *GetEntryAttributesResponse) Reset() { *m = GetEntryAttributesResponse{} } func (m *GetEntryAttributesResponse) Reset() { *m = GetEntryAttributesResponse{} }
func (m *GetEntryAttributesResponse) String() string { return proto.CompactTextString(m) } func (m *GetEntryAttributesResponse) String() string { return proto.CompactTextString(m) }
func (*GetEntryAttributesResponse) ProtoMessage() {} func (*GetEntryAttributesResponse) ProtoMessage() {}
func (*GetEntryAttributesResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } func (*GetEntryAttributesResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} }
func (m *GetEntryAttributesResponse) GetAttributes() *FuseAttributes { func (m *GetEntryAttributesResponse) GetAttributes() *FuseAttributes {
if m != nil { if m != nil {
@ -392,7 +425,7 @@ type GetFileContentRequest struct {
func (m *GetFileContentRequest) Reset() { *m = GetFileContentRequest{} } func (m *GetFileContentRequest) Reset() { *m = GetFileContentRequest{} }
func (m *GetFileContentRequest) String() string { return proto.CompactTextString(m) } func (m *GetFileContentRequest) String() string { return proto.CompactTextString(m) }
func (*GetFileContentRequest) ProtoMessage() {} func (*GetFileContentRequest) ProtoMessage() {}
func (*GetFileContentRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} } func (*GetFileContentRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} }
func (m *GetFileContentRequest) GetFileId() string { func (m *GetFileContentRequest) GetFileId() string {
if m != nil { if m != nil {
@ -408,7 +441,7 @@ type GetFileContentResponse struct {
func (m *GetFileContentResponse) Reset() { *m = GetFileContentResponse{} } func (m *GetFileContentResponse) Reset() { *m = GetFileContentResponse{} }
func (m *GetFileContentResponse) String() string { return proto.CompactTextString(m) } func (m *GetFileContentResponse) String() string { return proto.CompactTextString(m) }
func (*GetFileContentResponse) ProtoMessage() {} func (*GetFileContentResponse) ProtoMessage() {}
func (*GetFileContentResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } func (*GetFileContentResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} }
func (m *GetFileContentResponse) GetContent() []byte { func (m *GetFileContentResponse) GetContent() []byte {
if m != nil { if m != nil {
@ -425,7 +458,7 @@ type CreateEntryRequest struct {
func (m *CreateEntryRequest) Reset() { *m = CreateEntryRequest{} } func (m *CreateEntryRequest) Reset() { *m = CreateEntryRequest{} }
func (m *CreateEntryRequest) String() string { return proto.CompactTextString(m) } func (m *CreateEntryRequest) String() string { return proto.CompactTextString(m) }
func (*CreateEntryRequest) ProtoMessage() {} func (*CreateEntryRequest) ProtoMessage() {}
func (*CreateEntryRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} } func (*CreateEntryRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} }
func (m *CreateEntryRequest) GetDirectory() string { func (m *CreateEntryRequest) GetDirectory() string {
if m != nil { if m != nil {
@ -447,7 +480,7 @@ type CreateEntryResponse struct {
func (m *CreateEntryResponse) Reset() { *m = CreateEntryResponse{} } func (m *CreateEntryResponse) Reset() { *m = CreateEntryResponse{} }
func (m *CreateEntryResponse) String() string { return proto.CompactTextString(m) } func (m *CreateEntryResponse) String() string { return proto.CompactTextString(m) }
func (*CreateEntryResponse) ProtoMessage() {} func (*CreateEntryResponse) ProtoMessage() {}
func (*CreateEntryResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} } func (*CreateEntryResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} }
type UpdateEntryRequest struct { type UpdateEntryRequest struct {
Directory string `protobuf:"bytes,1,opt,name=directory" json:"directory,omitempty"` Directory string `protobuf:"bytes,1,opt,name=directory" json:"directory,omitempty"`
@ -457,7 +490,7 @@ type UpdateEntryRequest struct {
func (m *UpdateEntryRequest) Reset() { *m = UpdateEntryRequest{} } func (m *UpdateEntryRequest) Reset() { *m = UpdateEntryRequest{} }
func (m *UpdateEntryRequest) String() string { return proto.CompactTextString(m) } func (m *UpdateEntryRequest) String() string { return proto.CompactTextString(m) }
func (*UpdateEntryRequest) ProtoMessage() {} func (*UpdateEntryRequest) ProtoMessage() {}
func (*UpdateEntryRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} } func (*UpdateEntryRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{14} }
func (m *UpdateEntryRequest) GetDirectory() string { func (m *UpdateEntryRequest) GetDirectory() string {
if m != nil { if m != nil {
@ -479,7 +512,7 @@ type UpdateEntryResponse struct {
func (m *UpdateEntryResponse) Reset() { *m = UpdateEntryResponse{} } func (m *UpdateEntryResponse) Reset() { *m = UpdateEntryResponse{} }
func (m *UpdateEntryResponse) String() string { return proto.CompactTextString(m) } func (m *UpdateEntryResponse) String() string { return proto.CompactTextString(m) }
func (*UpdateEntryResponse) ProtoMessage() {} func (*UpdateEntryResponse) ProtoMessage() {}
func (*UpdateEntryResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{14} } func (*UpdateEntryResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{15} }
type DeleteEntryRequest struct { type DeleteEntryRequest struct {
Directory string `protobuf:"bytes,1,opt,name=directory" json:"directory,omitempty"` Directory string `protobuf:"bytes,1,opt,name=directory" json:"directory,omitempty"`
@ -492,7 +525,7 @@ type DeleteEntryRequest struct {
func (m *DeleteEntryRequest) Reset() { *m = DeleteEntryRequest{} } func (m *DeleteEntryRequest) Reset() { *m = DeleteEntryRequest{} }
func (m *DeleteEntryRequest) String() string { return proto.CompactTextString(m) } func (m *DeleteEntryRequest) String() string { return proto.CompactTextString(m) }
func (*DeleteEntryRequest) ProtoMessage() {} func (*DeleteEntryRequest) ProtoMessage() {}
func (*DeleteEntryRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{15} } func (*DeleteEntryRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{16} }
func (m *DeleteEntryRequest) GetDirectory() string { func (m *DeleteEntryRequest) GetDirectory() string {
if m != nil { if m != nil {
@ -535,7 +568,7 @@ type DeleteEntryResponse struct {
func (m *DeleteEntryResponse) Reset() { *m = DeleteEntryResponse{} } func (m *DeleteEntryResponse) Reset() { *m = DeleteEntryResponse{} }
func (m *DeleteEntryResponse) String() string { return proto.CompactTextString(m) } func (m *DeleteEntryResponse) String() string { return proto.CompactTextString(m) }
func (*DeleteEntryResponse) ProtoMessage() {} func (*DeleteEntryResponse) ProtoMessage() {}
func (*DeleteEntryResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{16} } func (*DeleteEntryResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17} }
type AssignVolumeRequest struct { type AssignVolumeRequest struct {
Count int32 `protobuf:"varint,1,opt,name=count" json:"count,omitempty"` Count int32 `protobuf:"varint,1,opt,name=count" json:"count,omitempty"`
@ -548,7 +581,7 @@ type AssignVolumeRequest struct {
func (m *AssignVolumeRequest) Reset() { *m = AssignVolumeRequest{} } func (m *AssignVolumeRequest) Reset() { *m = AssignVolumeRequest{} }
func (m *AssignVolumeRequest) String() string { return proto.CompactTextString(m) } func (m *AssignVolumeRequest) String() string { return proto.CompactTextString(m) }
func (*AssignVolumeRequest) ProtoMessage() {} func (*AssignVolumeRequest) ProtoMessage() {}
func (*AssignVolumeRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17} } func (*AssignVolumeRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{18} }
func (m *AssignVolumeRequest) GetCount() int32 { func (m *AssignVolumeRequest) GetCount() int32 {
if m != nil { if m != nil {
@ -595,7 +628,7 @@ type AssignVolumeResponse struct {
func (m *AssignVolumeResponse) Reset() { *m = AssignVolumeResponse{} } func (m *AssignVolumeResponse) Reset() { *m = AssignVolumeResponse{} }
func (m *AssignVolumeResponse) String() string { return proto.CompactTextString(m) } func (m *AssignVolumeResponse) String() string { return proto.CompactTextString(m) }
func (*AssignVolumeResponse) ProtoMessage() {} func (*AssignVolumeResponse) ProtoMessage() {}
func (*AssignVolumeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{18} } func (*AssignVolumeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{19} }
func (m *AssignVolumeResponse) GetFileId() string { func (m *AssignVolumeResponse) GetFileId() string {
if m != nil { if m != nil {
@ -632,7 +665,7 @@ type LookupVolumeRequest struct {
func (m *LookupVolumeRequest) Reset() { *m = LookupVolumeRequest{} } func (m *LookupVolumeRequest) Reset() { *m = LookupVolumeRequest{} }
func (m *LookupVolumeRequest) String() string { return proto.CompactTextString(m) } func (m *LookupVolumeRequest) String() string { return proto.CompactTextString(m) }
func (*LookupVolumeRequest) ProtoMessage() {} func (*LookupVolumeRequest) ProtoMessage() {}
func (*LookupVolumeRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{19} } func (*LookupVolumeRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20} }
func (m *LookupVolumeRequest) GetVolumeIds() []string { func (m *LookupVolumeRequest) GetVolumeIds() []string {
if m != nil { if m != nil {
@ -648,7 +681,7 @@ type Locations struct {
func (m *Locations) Reset() { *m = Locations{} } func (m *Locations) Reset() { *m = Locations{} }
func (m *Locations) String() string { return proto.CompactTextString(m) } func (m *Locations) String() string { return proto.CompactTextString(m) }
func (*Locations) ProtoMessage() {} func (*Locations) ProtoMessage() {}
func (*Locations) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20} } func (*Locations) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{21} }
func (m *Locations) GetLocations() []*Location { func (m *Locations) GetLocations() []*Location {
if m != nil { if m != nil {
@ -665,7 +698,7 @@ type Location struct {
func (m *Location) Reset() { *m = Location{} } func (m *Location) Reset() { *m = Location{} }
func (m *Location) String() string { return proto.CompactTextString(m) } func (m *Location) String() string { return proto.CompactTextString(m) }
func (*Location) ProtoMessage() {} func (*Location) ProtoMessage() {}
func (*Location) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{21} } func (*Location) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{22} }
func (m *Location) GetUrl() string { func (m *Location) GetUrl() string {
if m != nil { if m != nil {
@ -688,7 +721,7 @@ type LookupVolumeResponse struct {
func (m *LookupVolumeResponse) Reset() { *m = LookupVolumeResponse{} } func (m *LookupVolumeResponse) Reset() { *m = LookupVolumeResponse{} }
func (m *LookupVolumeResponse) String() string { return proto.CompactTextString(m) } func (m *LookupVolumeResponse) String() string { return proto.CompactTextString(m) }
func (*LookupVolumeResponse) ProtoMessage() {} func (*LookupVolumeResponse) ProtoMessage() {}
func (*LookupVolumeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{22} } func (*LookupVolumeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{23} }
func (m *LookupVolumeResponse) GetLocationsMap() map[string]*Locations { func (m *LookupVolumeResponse) GetLocationsMap() map[string]*Locations {
if m != nil { if m != nil {
@ -704,7 +737,7 @@ type DeleteCollectionRequest struct {
func (m *DeleteCollectionRequest) Reset() { *m = DeleteCollectionRequest{} } func (m *DeleteCollectionRequest) Reset() { *m = DeleteCollectionRequest{} }
func (m *DeleteCollectionRequest) String() string { return proto.CompactTextString(m) } func (m *DeleteCollectionRequest) String() string { return proto.CompactTextString(m) }
func (*DeleteCollectionRequest) ProtoMessage() {} func (*DeleteCollectionRequest) ProtoMessage() {}
func (*DeleteCollectionRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{23} } func (*DeleteCollectionRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{24} }
func (m *DeleteCollectionRequest) GetCollection() string { func (m *DeleteCollectionRequest) GetCollection() string {
if m != nil { if m != nil {
@ -719,7 +752,7 @@ type DeleteCollectionResponse struct {
func (m *DeleteCollectionResponse) Reset() { *m = DeleteCollectionResponse{} } func (m *DeleteCollectionResponse) Reset() { *m = DeleteCollectionResponse{} }
func (m *DeleteCollectionResponse) String() string { return proto.CompactTextString(m) } func (m *DeleteCollectionResponse) String() string { return proto.CompactTextString(m) }
func (*DeleteCollectionResponse) ProtoMessage() {} func (*DeleteCollectionResponse) ProtoMessage() {}
func (*DeleteCollectionResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{24} } func (*DeleteCollectionResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{25} }
func init() { func init() {
proto.RegisterType((*LookupDirectoryEntryRequest)(nil), "filer_pb.LookupDirectoryEntryRequest") proto.RegisterType((*LookupDirectoryEntryRequest)(nil), "filer_pb.LookupDirectoryEntryRequest")
@ -727,6 +760,7 @@ func init() {
proto.RegisterType((*ListEntriesRequest)(nil), "filer_pb.ListEntriesRequest") proto.RegisterType((*ListEntriesRequest)(nil), "filer_pb.ListEntriesRequest")
proto.RegisterType((*ListEntriesResponse)(nil), "filer_pb.ListEntriesResponse") proto.RegisterType((*ListEntriesResponse)(nil), "filer_pb.ListEntriesResponse")
proto.RegisterType((*Entry)(nil), "filer_pb.Entry") proto.RegisterType((*Entry)(nil), "filer_pb.Entry")
proto.RegisterType((*EventNotification)(nil), "filer_pb.EventNotification")
proto.RegisterType((*FileChunk)(nil), "filer_pb.FileChunk") proto.RegisterType((*FileChunk)(nil), "filer_pb.FileChunk")
proto.RegisterType((*FuseAttributes)(nil), "filer_pb.FuseAttributes") proto.RegisterType((*FuseAttributes)(nil), "filer_pb.FuseAttributes")
proto.RegisterType((*GetEntryAttributesRequest)(nil), "filer_pb.GetEntryAttributesRequest") proto.RegisterType((*GetEntryAttributesRequest)(nil), "filer_pb.GetEntryAttributesRequest")
@ -1088,75 +1122,78 @@ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
func init() { proto.RegisterFile("filer.proto", fileDescriptor0) } func init() { proto.RegisterFile("filer.proto", fileDescriptor0) }
var fileDescriptor0 = []byte{ var fileDescriptor0 = []byte{
// 1108 bytes of a gzipped FileDescriptorProto // 1154 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x57, 0x4f, 0x6f, 0xdc, 0x44, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x57, 0xcd, 0x6e, 0xdb, 0x46,
0x14, 0x8f, 0xd7, 0xbb, 0x9b, 0xf8, 0xed, 0xa6, 0xb4, 0xb3, 0x69, 0x6b, 0xb6, 0x49, 0xd9, 0x0e, 0x10, 0x36, 0x25, 0x4b, 0x36, 0x47, 0x72, 0x1a, 0xaf, 0x9c, 0x84, 0x55, 0xec, 0x54, 0xd9, 0x26,
0x2d, 0x4a, 0x05, 0x8a, 0xa2, 0xc0, 0xa1, 0x80, 0x90, 0xa8, 0x92, 0xb6, 0xaa, 0x94, 0x52, 0xc9, 0x85, 0x83, 0x1a, 0x86, 0xe1, 0xf6, 0x90, 0xb6, 0x28, 0xd0, 0xc0, 0x76, 0x82, 0x00, 0x4e, 0x02,
0x21, 0x48, 0x88, 0xc3, 0xca, 0xb1, 0x67, 0xc3, 0x28, 0x5e, 0xdb, 0xcc, 0x8c, 0x03, 0xe1, 0xca, 0xd0, 0x75, 0x81, 0xa2, 0x07, 0x81, 0x26, 0x47, 0xee, 0xc2, 0x14, 0xa9, 0x72, 0x97, 0x76, 0xdd,
0x91, 0x13, 0x1f, 0x02, 0x71, 0xe7, 0x03, 0x70, 0xe1, 0x8b, 0xa1, 0xf9, 0x63, 0xef, 0x78, 0xed, 0x6b, 0x8f, 0x3d, 0xf5, 0x21, 0x8a, 0xde, 0xfb, 0x00, 0xbd, 0xf4, 0xc5, 0x8a, 0xfd, 0xa1, 0xb4,
0x4d, 0xdb, 0x03, 0xb7, 0x99, 0xf7, 0xde, 0xfc, 0xe6, 0xf7, 0xde, 0xbc, 0xf7, 0xf3, 0x2e, 0x0c, 0x14, 0x29, 0x27, 0x39, 0xe4, 0xb6, 0x3b, 0x33, 0x3b, 0x33, 0xdf, 0xec, 0xcc, 0xc7, 0x25, 0x74,
0x66, 0x34, 0x21, 0x6c, 0x2f, 0x67, 0x99, 0xc8, 0xd0, 0x86, 0xda, 0x4c, 0xf3, 0x33, 0xfc, 0x1a, 0x46, 0x2c, 0xc6, 0x6c, 0x77, 0x92, 0xa5, 0x22, 0x25, 0xab, 0x6a, 0x33, 0x9c, 0x9c, 0xd1, 0x37,
0xee, 0x1d, 0x67, 0xd9, 0x45, 0x91, 0x1f, 0x51, 0x46, 0x22, 0x91, 0xb1, 0xab, 0x67, 0xa9, 0x60, 0x70, 0xff, 0x38, 0x4d, 0x2f, 0xf2, 0xc9, 0x21, 0xcb, 0x30, 0x14, 0x69, 0x76, 0x7d, 0x94, 0x88,
0x57, 0x01, 0xf9, 0xa9, 0x20, 0x5c, 0xa0, 0x6d, 0xf0, 0xe2, 0xd2, 0xe1, 0x3b, 0x13, 0x67, 0xd7, 0xec, 0xda, 0xc7, 0x5f, 0x72, 0xe4, 0x82, 0x6c, 0x82, 0x1b, 0x15, 0x0a, 0xcf, 0x19, 0x38, 0xdb,
0x0b, 0x16, 0x06, 0x84, 0xa0, 0x9b, 0x86, 0x73, 0xe2, 0x77, 0x94, 0x43, 0xad, 0xf1, 0x33, 0xd8, 0xae, 0x3f, 0x13, 0x10, 0x02, 0xcb, 0x49, 0x30, 0x46, 0xaf, 0xa1, 0x14, 0x6a, 0x4d, 0x8f, 0x60,
0x6e, 0x07, 0xe4, 0x79, 0x96, 0x72, 0x82, 0x1e, 0x41, 0x8f, 0x48, 0x83, 0x42, 0x1b, 0x1c, 0xbc, 0xb3, 0xde, 0x21, 0x9f, 0xa4, 0x09, 0x47, 0xf2, 0x18, 0x5a, 0x28, 0x05, 0xca, 0x5b, 0x67, 0xff,
0xb7, 0x57, 0x52, 0xd9, 0xd3, 0x71, 0xda, 0x8b, 0xff, 0x71, 0x00, 0x1d, 0x53, 0x2e, 0xa4, 0x91, 0xa3, 0xdd, 0x22, 0x95, 0x5d, 0x6d, 0xa7, 0xb5, 0xf4, 0x5f, 0x07, 0xc8, 0x31, 0xe3, 0x42, 0x0a,
0x12, 0xfe, 0x76, 0x7c, 0xee, 0x40, 0x3f, 0x67, 0x64, 0x46, 0x7f, 0x31, 0x8c, 0xcc, 0x0e, 0x7d, 0x19, 0xf2, 0x77, 0xcb, 0xe7, 0x2e, 0xb4, 0x27, 0x19, 0x8e, 0xd8, 0xaf, 0x26, 0x23, 0xb3, 0x23,
0x02, 0xb7, 0xb8, 0x08, 0x99, 0x78, 0xce, 0xb2, 0xf9, 0x73, 0x9a, 0x90, 0x6f, 0x24, 0x69, 0x57, 0x3b, 0xb0, 0xce, 0x45, 0x90, 0x89, 0xe7, 0x59, 0x3a, 0x7e, 0xce, 0x62, 0x7c, 0x2d, 0x93, 0x6e,
0x85, 0x34, 0x1d, 0x68, 0x0f, 0x10, 0x4d, 0xa3, 0xa4, 0xe0, 0xf4, 0x92, 0x9c, 0x94, 0x5e, 0xbf, 0x2a, 0x93, 0xaa, 0x82, 0xec, 0x02, 0x61, 0x49, 0x18, 0xe7, 0x9c, 0x5d, 0xe2, 0x49, 0xa1, 0xf5,
0x3b, 0x71, 0x76, 0x37, 0x82, 0x16, 0x0f, 0xda, 0x82, 0x5e, 0x42, 0xe7, 0x54, 0xf8, 0xbd, 0x89, 0x96, 0x07, 0xce, 0xf6, 0xaa, 0x5f, 0xa3, 0x21, 0x1b, 0xd0, 0x8a, 0xd9, 0x98, 0x09, 0xaf, 0x35,
0xb3, 0xbb, 0x19, 0xe8, 0x0d, 0xfe, 0x1a, 0x46, 0x35, 0xfe, 0x26, 0xfd, 0xc7, 0xb0, 0x4e, 0xb4, 0x70, 0xb6, 0xd7, 0x7c, 0xbd, 0xa1, 0xdf, 0x41, 0xaf, 0x94, 0xbf, 0x81, 0xff, 0x04, 0x56, 0x50,
0xc9, 0x77, 0x26, 0x6e, 0x5b, 0x01, 0x4a, 0x3f, 0xfe, 0xd3, 0x81, 0x9e, 0x32, 0x55, 0x75, 0x76, 0x8b, 0x3c, 0x67, 0xd0, 0xac, 0x2b, 0x40, 0xa1, 0xa7, 0x7f, 0x39, 0xd0, 0x52, 0xa2, 0x69, 0x9d,
0x16, 0x75, 0x46, 0x0f, 0x60, 0x48, 0xf9, 0x74, 0x51, 0x8c, 0x8e, 0xe2, 0x37, 0xa0, 0xbc, 0xaa, 0x9d, 0x59, 0x9d, 0xc9, 0x43, 0xe8, 0x32, 0x3e, 0x9c, 0x15, 0xa3, 0xa1, 0xf2, 0xeb, 0x30, 0x3e,
0x3b, 0xfa, 0x18, 0xfa, 0xd1, 0x8f, 0x45, 0x7a, 0xc1, 0x7d, 0x57, 0x5d, 0x35, 0x5a, 0x5c, 0x25, 0xad, 0x3b, 0xf9, 0x1c, 0xda, 0xe1, 0xcf, 0x79, 0x72, 0xc1, 0xbd, 0xa6, 0x0a, 0xd5, 0x9b, 0x85,
0x93, 0x3d, 0x94, 0xbe, 0xc0, 0x84, 0xa0, 0x27, 0x00, 0xa1, 0x10, 0x8c, 0x9e, 0x15, 0x82, 0x70, 0x92, 0x60, 0x0f, 0xa4, 0xce, 0x37, 0x26, 0xe4, 0x29, 0x40, 0x20, 0x44, 0xc6, 0xce, 0x72, 0x81,
0x95, 0xed, 0xe0, 0xc0, 0xb7, 0x0e, 0x14, 0x9c, 0x3c, 0xad, 0xfc, 0x81, 0x15, 0x8b, 0x67, 0xe0, 0x5c, 0xa1, 0xed, 0xec, 0x7b, 0xd6, 0x81, 0x9c, 0xe3, 0xb3, 0xa9, 0xde, 0xb7, 0x6c, 0xe9, 0xef,
0x55, 0x70, 0xe8, 0x2e, 0xac, 0xcb, 0x33, 0x53, 0x1a, 0x1b, 0xb6, 0x7d, 0xb9, 0x7d, 0x19, 0xcb, 0x0e, 0xac, 0x1f, 0x5d, 0x62, 0x22, 0x5e, 0xa7, 0x82, 0x8d, 0x58, 0x18, 0x08, 0x96, 0x26, 0xb5,
0xb7, 0xc9, 0x66, 0x33, 0x4e, 0x84, 0x62, 0xea, 0x06, 0x66, 0x27, 0x73, 0xe3, 0xf4, 0x57, 0xfd, 0x39, 0xef, 0x80, 0x9b, 0xc6, 0xd1, 0x50, 0xdf, 0x7f, 0xa3, 0xfe, 0xfe, 0x57, 0xd3, 0x38, 0xd2,
0x1c, 0xdd, 0x40, 0xad, 0x65, 0x45, 0xe7, 0x82, 0xce, 0x89, 0xa2, 0xe1, 0x06, 0x7a, 0x83, 0x7f, 0xa8, 0x77, 0xc0, 0x4d, 0xf0, 0xca, 0x58, 0x37, 0x17, 0x58, 0x27, 0x78, 0xa5, 0x56, 0x74, 0x04,
0xef, 0xc0, 0x8d, 0x3a, 0x0d, 0x74, 0x0f, 0x3c, 0x75, 0x9b, 0x42, 0x70, 0x14, 0x82, 0x6a, 0xed, 0xee, 0x14, 0x14, 0xb9, 0x07, 0x2b, 0xd2, 0x70, 0xc8, 0x22, 0x13, 0xbf, 0x2d, 0xb7, 0x2f, 0x23,
0x93, 0x1a, 0x4a, 0xc7, 0x42, 0xa9, 0x8e, 0xcc, 0xb3, 0x58, 0x5f, 0xba, 0xa9, 0x8f, 0xbc, 0xca, 0xd9, 0x21, 0xe9, 0x68, 0xc4, 0x51, 0xa8, 0xf0, 0x4d, 0xdf, 0xec, 0x64, 0xb6, 0x9c, 0xfd, 0xa6,
0x62, 0x82, 0x6e, 0x82, 0x5b, 0xd0, 0x58, 0x5d, 0xbb, 0x19, 0xc8, 0xa5, 0xb4, 0x9c, 0xd3, 0xd8, 0x9b, 0x62, 0xd9, 0x57, 0x6b, 0x79, 0xaf, 0x63, 0xc1, 0xc6, 0xa8, 0x8a, 0xd1, 0xf4, 0xf5, 0x86,
0x3c, 0xad, 0x5c, 0xca, 0x44, 0x22, 0xa6, 0x70, 0xfb, 0x3a, 0x11, 0xbd, 0x93, 0x89, 0xcc, 0xa5, 0xfe, 0xd1, 0x80, 0x5b, 0xe5, 0x62, 0x90, 0xfb, 0xe0, 0xaa, 0x68, 0xca, 0x83, 0xa3, 0x3c, 0xa8,
0x75, 0x5d, 0x3f, 0x92, 0x5c, 0xa3, 0x09, 0x0c, 0x18, 0xc9, 0x13, 0x1a, 0x85, 0x82, 0x66, 0xa9, 0x01, 0x3b, 0x29, 0x79, 0x69, 0x58, 0x5e, 0xa6, 0x47, 0xc6, 0x69, 0xa4, 0x83, 0xae, 0xe9, 0x23,
0xbf, 0xa1, 0x5c, 0xb6, 0x09, 0xdd, 0x07, 0x88, 0xb2, 0x24, 0x21, 0x91, 0x0a, 0xf0, 0x54, 0x80, 0xaf, 0xd2, 0x08, 0xc9, 0x6d, 0x68, 0xe6, 0x2c, 0x52, 0x61, 0xd7, 0x7c, 0xb9, 0x94, 0x92, 0x73,
0x65, 0x91, 0xf5, 0x14, 0x22, 0x99, 0x72, 0x12, 0xf9, 0x30, 0x71, 0x76, 0x7b, 0x41, 0x5f, 0x88, 0x16, 0x99, 0x06, 0x93, 0x4b, 0x09, 0x24, 0xcc, 0x94, 0xdf, 0xb6, 0x06, 0xa2, 0x77, 0x12, 0xc8,
0xe4, 0x84, 0x44, 0xf8, 0x1c, 0xde, 0x7f, 0x41, 0x54, 0x7b, 0x5d, 0x59, 0xef, 0x62, 0xc6, 0xa4, 0x58, 0x4a, 0x57, 0x74, 0xd9, 0xe5, 0x9a, 0x0c, 0xa0, 0x93, 0xe1, 0x24, 0x36, 0x37, 0xe3, 0xad,
0xad, 0x61, 0x76, 0x00, 0xf2, 0x90, 0x91, 0x54, 0xc8, 0xa6, 0x31, 0x03, 0xe2, 0x69, 0xcb, 0x11, 0x2a, 0x95, 0x2d, 0x22, 0x0f, 0x00, 0xc2, 0x34, 0x8e, 0x31, 0x54, 0x06, 0xae, 0x32, 0xb0, 0x24,
0x65, 0xf6, 0xc3, 0xb9, 0xf6, 0xc3, 0xe1, 0xdf, 0x1c, 0x18, 0xb7, 0xdd, 0x64, 0x1a, 0xba, 0xde, 0xb2, 0x9e, 0x42, 0xc4, 0x43, 0x8e, 0xa1, 0x07, 0x03, 0x67, 0xbb, 0xe5, 0xb7, 0x85, 0x88, 0x4f,
0x37, 0xce, 0xdb, 0xf7, 0x8d, 0xd5, 0x9e, 0x9d, 0x37, 0xb6, 0x27, 0xde, 0x87, 0xdb, 0x2f, 0x88, 0x30, 0xa4, 0xe7, 0xf0, 0xf1, 0x0b, 0x54, 0x4d, 0x7e, 0x6d, 0x75, 0x87, 0x19, 0xd6, 0xba, 0x16,
0x50, 0xf6, 0x2c, 0x15, 0x24, 0x15, 0x65, 0xaa, 0xab, 0x1a, 0x0e, 0x1f, 0xc0, 0x9d, 0xe5, 0x13, 0xd8, 0x02, 0x98, 0x04, 0x19, 0x26, 0x42, 0xb6, 0xae, 0x19, 0x53, 0x57, 0x4b, 0x0e, 0x59, 0x66,
0x86, 0xb2, 0x0f, 0xeb, 0x91, 0x36, 0xa9, 0x23, 0xc3, 0xa0, 0xdc, 0xe2, 0xef, 0x01, 0x1d, 0x32, 0x5f, 0x5c, 0xd3, 0xbe, 0x38, 0xd9, 0x64, 0xfd, 0xba, 0x48, 0x66, 0xac, 0xca, 0xdd, 0xeb, 0xbc,
0x12, 0x0a, 0xf2, 0x0e, 0x22, 0x58, 0x09, 0x5a, 0xe7, 0x5a, 0x41, 0xbb, 0x0d, 0xa3, 0x1a, 0xb4, 0x7b, 0xf7, 0x5a, 0x43, 0xd2, 0x78, 0xeb, 0x90, 0xd0, 0x3d, 0xb8, 0xf3, 0x02, 0x85, 0x92, 0xa7,
0xe6, 0x22, 0x6f, 0x3c, 0xcd, 0xe3, 0xff, 0xeb, 0xc6, 0x1a, 0xb4, 0xb9, 0xf1, 0x6f, 0x07, 0xd0, 0x89, 0xc0, 0x44, 0x14, 0x50, 0x17, 0x35, 0x1c, 0xdd, 0x87, 0xbb, 0xf3, 0x27, 0x4c, 0xca, 0x1e,
0x11, 0x49, 0xc8, 0x3b, 0x5d, 0xd9, 0xa2, 0xf4, 0x0d, 0x05, 0x72, 0x9b, 0x0a, 0xf4, 0x10, 0x6e, 0xac, 0x84, 0x5a, 0xa4, 0x8e, 0x74, 0xfd, 0x62, 0x4b, 0x7f, 0x04, 0x72, 0x90, 0x61, 0x20, 0xf0,
0xc8, 0x10, 0x75, 0xdb, 0x34, 0x0e, 0x45, 0x68, 0x64, 0x74, 0x48, 0xb9, 0xa6, 0x70, 0x14, 0x8a, 0x3d, 0xa8, 0x78, 0x4a, 0xab, 0x8d, 0x1b, 0x69, 0xf5, 0x0e, 0xf4, 0x4a, 0xae, 0x75, 0x2e, 0x32,
0xd0, 0x00, 0x31, 0x12, 0x15, 0x4c, 0x2a, 0xab, 0x1a, 0x36, 0x05, 0x14, 0x94, 0x26, 0x99, 0x4b, 0xe2, 0xe9, 0x24, 0xfa, 0x50, 0x11, 0x4b, 0xae, 0x4d, 0xc4, 0x7f, 0x1c, 0x20, 0x87, 0x18, 0xe3,
0x8d, 0xb3, 0xc9, 0xe5, 0x2f, 0x07, 0x46, 0x4f, 0x39, 0xa7, 0xe7, 0xe9, 0x77, 0x59, 0x52, 0xcc, 0x7b, 0x85, 0xac, 0xf9, 0xde, 0x54, 0x78, 0xb0, 0x59, 0xe5, 0xc1, 0x47, 0x70, 0x4b, 0x9a, 0xa8,
0x49, 0x99, 0xcc, 0x16, 0xf4, 0xa2, 0xac, 0x30, 0xef, 0xdb, 0x0b, 0xf4, 0x66, 0x69, 0xd6, 0x3a, 0x68, 0xc3, 0x28, 0x10, 0x81, 0x21, 0xf3, 0x2e, 0xe3, 0x3a, 0x85, 0xc3, 0x40, 0x04, 0xc6, 0x51,
0x8d, 0x59, 0x5b, 0x9a, 0x56, 0xb7, 0x39, 0xad, 0xd6, 0x34, 0x76, 0xed, 0x69, 0x44, 0x1f, 0xc0, 0x86, 0x61, 0x9e, 0x49, 0x7e, 0x57, 0xc3, 0xa6, 0x1c, 0xf9, 0x85, 0x48, 0x62, 0x29, 0xe5, 0x6c,
0x40, 0xa6, 0x37, 0x8d, 0x48, 0x2a, 0x08, 0x53, 0x19, 0x78, 0x01, 0x48, 0xd3, 0xa1, 0xb2, 0xe0, 0xb0, 0xfc, 0xed, 0x40, 0xef, 0x19, 0xe7, 0xec, 0x3c, 0xf9, 0x21, 0x8d, 0xf3, 0x31, 0x16, 0x60,
0x4b, 0xd8, 0xaa, 0x13, 0x35, 0xbd, 0xb8, 0x52, 0x2f, 0xa5, 0x14, 0xb1, 0xc4, 0xb0, 0x94, 0x4b, 0x36, 0xa0, 0x15, 0xa6, 0xb9, 0xb9, 0xdf, 0x96, 0xaf, 0x37, 0x73, 0xb3, 0xd6, 0xa8, 0xcc, 0xda,
0x35, 0xc0, 0xc5, 0x59, 0x42, 0xa3, 0xa9, 0x74, 0xb8, 0x66, 0x80, 0x95, 0xe5, 0x94, 0x25, 0x8b, 0xdc, 0xb4, 0x36, 0xab, 0xd3, 0x6a, 0x4d, 0xe3, 0xb2, 0x3d, 0x8d, 0xe4, 0x13, 0xe8, 0x48, 0x78,
0x9c, 0xbb, 0x56, 0xce, 0xf8, 0x33, 0x18, 0xe9, 0xcf, 0x71, 0xbd, 0x40, 0x3b, 0x00, 0x97, 0xca, 0xc3, 0x10, 0x13, 0x81, 0x99, 0x42, 0xe0, 0xfa, 0x20, 0x45, 0x07, 0x4a, 0x42, 0x2f, 0x61, 0xa3,
0x30, 0xa5, 0xb1, 0xfe, 0x12, 0x79, 0x81, 0xa7, 0x2d, 0x2f, 0x63, 0x8e, 0xbf, 0x02, 0xef, 0x38, 0x9c, 0xa8, 0xe9, 0xc5, 0x85, 0x7c, 0x29, 0xa9, 0x28, 0x8b, 0x4d, 0x96, 0x72, 0xa9, 0x06, 0x38,
0xd3, 0x39, 0x73, 0xb4, 0x0f, 0x5e, 0x52, 0x6e, 0xcc, 0x47, 0x0b, 0x2d, 0x5a, 0xae, 0x8c, 0x0b, 0x3f, 0x8b, 0x59, 0x38, 0x94, 0x8a, 0xa6, 0x19, 0x60, 0x25, 0x39, 0xcd, 0xe2, 0x19, 0xe6, 0x65,
0x16, 0x41, 0xf8, 0x4b, 0xd8, 0x28, 0xcd, 0x65, 0x1e, 0xce, 0xaa, 0x3c, 0x3a, 0x4b, 0x79, 0xe0, 0x0b, 0x33, 0xfd, 0x12, 0x7a, 0xfa, 0x51, 0x50, 0x2e, 0xd0, 0x16, 0xc0, 0xa5, 0x12, 0x0c, 0x59,
0x7f, 0x1d, 0xd8, 0xaa, 0x53, 0x36, 0xa5, 0x3a, 0x85, 0xcd, 0xea, 0x8a, 0xe9, 0x3c, 0xcc, 0x0d, 0xa4, 0xbf, 0x87, 0xae, 0xef, 0x6a, 0xc9, 0xcb, 0x88, 0xd3, 0x6f, 0xc1, 0x3d, 0x4e, 0x35, 0x66,
0x97, 0x7d, 0x9b, 0x4b, 0xf3, 0x58, 0x45, 0x90, 0xbf, 0x0a, 0x73, 0xdd, 0x3d, 0xc3, 0xc4, 0x32, 0x4e, 0xf6, 0xc0, 0x8d, 0x8b, 0x8d, 0xf9, 0x74, 0x92, 0x59, 0xcb, 0x15, 0x76, 0xfe, 0xcc, 0x88,
0x8d, 0xbf, 0x85, 0x5b, 0x8d, 0x10, 0xc9, 0xfa, 0x82, 0x94, 0x73, 0x20, 0x97, 0xe8, 0x31, 0xf4, 0x7e, 0x03, 0xab, 0x85, 0xb8, 0xc0, 0xe1, 0x2c, 0xc2, 0xd1, 0x98, 0xc3, 0x41, 0xff, 0x73, 0x60,
0x2e, 0xc3, 0xa4, 0x20, 0x66, 0xe8, 0x46, 0xcd, 0x0a, 0xf0, 0x40, 0x47, 0x7c, 0xd1, 0x79, 0xe2, 0xa3, 0x9c, 0xb2, 0x29, 0xd5, 0x29, 0xac, 0x4d, 0x43, 0x0c, 0xc7, 0xc1, 0xc4, 0xe4, 0xb2, 0x67,
0xe0, 0xcf, 0xe1, 0xae, 0x6e, 0xd8, 0xc3, 0xaa, 0xbf, 0xca, 0xda, 0xd7, 0xdb, 0xd0, 0x59, 0x6e, 0xe7, 0x52, 0x3d, 0x36, 0x4d, 0x90, 0xbf, 0x0a, 0x26, 0xba, 0x7b, 0xba, 0xb1, 0x25, 0xea, 0x7f,
0x43, 0x3c, 0x06, 0xbf, 0x79, 0x54, 0x27, 0x73, 0xf0, 0x47, 0x1f, 0x86, 0x27, 0x24, 0xfc, 0x99, 0x0f, 0xeb, 0x15, 0x13, 0x99, 0xf5, 0x05, 0x16, 0x73, 0x20, 0x97, 0xe4, 0x09, 0xb4, 0x2e, 0x83,
0x90, 0x58, 0x2a, 0x1b, 0x43, 0xe7, 0x65, 0xb1, 0xea, 0x3f, 0xb7, 0xd0, 0xa3, 0xe5, 0xaa, 0xb4, 0x38, 0x47, 0x33, 0x74, 0xbd, 0x6a, 0x05, 0xb8, 0xaf, 0x2d, 0xbe, 0x6e, 0x3c, 0x75, 0xe8, 0x57,
0xfe, 0xbe, 0x1b, 0x7f, 0xf4, 0xa6, 0x30, 0x33, 0x68, 0x6b, 0xe8, 0x18, 0x06, 0xd6, 0xef, 0x19, 0x70, 0x4f, 0x37, 0xec, 0xc1, 0xb4, 0xbf, 0x8a, 0xda, 0x97, 0xdb, 0xd0, 0x99, 0x6f, 0x43, 0xda,
0xb4, 0x6d, 0x1d, 0x6c, 0xfc, 0x4c, 0x1b, 0xef, 0xac, 0xf0, 0x56, 0x68, 0x21, 0xa0, 0xe6, 0x37, 0x07, 0xaf, 0x7a, 0x54, 0x83, 0xd9, 0xff, 0xb3, 0x0d, 0xdd, 0x13, 0x0c, 0xae, 0x10, 0x23, 0xc9,
0x05, 0x7d, 0xb8, 0x38, 0xb6, 0xf2, 0xdb, 0x36, 0x7e, 0x78, 0x7d, 0x90, 0x4d, 0xd8, 0x12, 0x5c, 0x6c, 0x19, 0x39, 0x2f, 0x8a, 0x55, 0x7e, 0xf4, 0x91, 0xc7, 0xf3, 0x55, 0xa9, 0x7d, 0x65, 0xf6,
0x9b, 0x70, 0x53, 0xe2, 0x6d, 0xc2, 0x6d, 0x2a, 0xad, 0xd0, 0x2c, 0x31, 0xb5, 0xd1, 0x9a, 0xf2, 0x3f, 0x7b, 0x9b, 0x99, 0x19, 0xb4, 0x25, 0x72, 0x0c, 0x1d, 0xeb, 0x55, 0x45, 0x36, 0xad, 0x83,
0x6d, 0xa3, 0xb5, 0x29, 0xb0, 0x42, 0xb3, 0xe4, 0xcc, 0x46, 0x6b, 0x2a, 0xb3, 0x8d, 0xd6, 0xa6, 0x95, 0xc7, 0x62, 0x7f, 0x6b, 0x81, 0x76, 0xea, 0x2d, 0x00, 0x52, 0xfd, 0xa6, 0x90, 0x4f, 0x67,
0x81, 0x6b, 0xe8, 0x35, 0x0c, 0x6d, 0x6d, 0x41, 0xd6, 0x81, 0x16, 0x71, 0x1c, 0xdf, 0x5f, 0xe5, 0xc7, 0x16, 0x7e, 0xdb, 0xfa, 0x8f, 0x6e, 0x36, 0xb2, 0x13, 0xb6, 0x08, 0xd7, 0x4e, 0xb8, 0x4a,
0xb6, 0x01, 0xed, 0x51, 0xb2, 0x01, 0x5b, 0xc4, 0xc4, 0x06, 0x6c, 0x9b, 0x40, 0xbc, 0x86, 0x7e, 0xf1, 0x76, 0xc2, 0x75, 0x2c, 0xad, 0xbc, 0x59, 0x64, 0x6a, 0x7b, 0xab, 0xd2, 0xb7, 0xed, 0xad,
0x80, 0x9b, 0xcb, 0x2d, 0x8d, 0x1e, 0x2c, 0xa7, 0xd5, 0x98, 0x94, 0x31, 0xbe, 0x2e, 0xa4, 0x04, 0x8e, 0x81, 0x95, 0x37, 0x8b, 0xce, 0x6c, 0x6f, 0x55, 0x66, 0xb6, 0xbd, 0xd5, 0x71, 0xe0, 0x12,
0x3f, 0xeb, 0xab, 0xff, 0x34, 0x9f, 0xfe, 0x17, 0x00, 0x00, 0xff, 0xff, 0x41, 0x12, 0xff, 0xda, 0x79, 0x03, 0x5d, 0x9b, 0x5b, 0x88, 0x75, 0xa0, 0x86, 0x1c, 0xfb, 0x0f, 0x16, 0xa9, 0x6d, 0x87,
0xe2, 0x0c, 0x00, 0x00, 0xf6, 0x28, 0xd9, 0x0e, 0x6b, 0xc8, 0xc4, 0x76, 0x58, 0x37, 0x81, 0x74, 0x89, 0xfc, 0x04, 0xb7,
0xe7, 0x5b, 0x9a, 0x3c, 0x9c, 0x87, 0x55, 0x99, 0x94, 0x3e, 0xbd, 0xc9, 0xa4, 0x70, 0x7e, 0xd6,
0x56, 0x7f, 0x56, 0x5f, 0xfc, 0x1f, 0x00, 0x00, 0xff, 0xff, 0x41, 0x0a, 0x10, 0x9d, 0x68, 0x0d,
0x00, 0x00,
} }

View file

@ -173,6 +173,10 @@ func (fs *FilerServer) UpdateEntry(ctx context.Context, req *filer_pb.UpdateEntr
} }
if filer2.EqualEntry(entry, newEntry) {
return &filer_pb.UpdateEntryResponse{}, err
}
if err = fs.filer.UpdateEntry(newEntry); err == nil { if err = fs.filer.UpdateEntry(newEntry); err == nil {
for _, garbage := range unusedChunks { for _, garbage := range unusedChunks {
glog.V(0).Infof("deleting %s old chunk: %v, [%d, %d)", fullpath, garbage.FileId, garbage.Offset, garbage.Offset+int64(garbage.Size)) glog.V(0).Infof("deleting %s old chunk: %v, [%d, %d)", fullpath, garbage.FileId, garbage.Offset, garbage.Offset+int64(garbage.Size))
@ -184,6 +188,8 @@ func (fs *FilerServer) UpdateEntry(ctx context.Context, req *filer_pb.UpdateEntr
} }
} }
fs.filer.NotifyUpdateEvent(entry, newEntry)
return &filer_pb.UpdateEntryResponse{}, err return &filer_pb.UpdateEntryResponse{}, err
} }

View file

@ -8,9 +8,12 @@ import (
_ "github.com/chrislusf/seaweedfs/weed/filer2/mysql" _ "github.com/chrislusf/seaweedfs/weed/filer2/mysql"
_ "github.com/chrislusf/seaweedfs/weed/filer2/postgres" _ "github.com/chrislusf/seaweedfs/weed/filer2/postgres"
_ "github.com/chrislusf/seaweedfs/weed/filer2/redis" _ "github.com/chrislusf/seaweedfs/weed/filer2/redis"
_ "github.com/chrislusf/seaweedfs/weed/msgqueue/kafka"
_ "github.com/chrislusf/seaweedfs/weed/msgqueue/log"
"github.com/chrislusf/seaweedfs/weed/glog" "github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/security" "github.com/chrislusf/seaweedfs/weed/security"
"net/http" "net/http"
"github.com/chrislusf/seaweedfs/weed/msgqueue"
) )
type FilerOption struct { type FilerOption struct {
@ -23,6 +26,7 @@ type FilerOption struct {
SecretKey string SecretKey string
DirListingLimit int DirListingLimit int
DataCenter string DataCenter string
EnableNotification bool
} }
type FilerServer struct { type FilerServer struct {
@ -46,6 +50,10 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
fs.filer.LoadConfiguration() fs.filer.LoadConfiguration()
if fs.option.EnableNotification {
msgqueue.LoadConfiguration()
}
defaultMux.HandleFunc("/favicon.ico", faviconHandler) defaultMux.HandleFunc("/favicon.ico", faviconHandler)
defaultMux.HandleFunc("/", fs.filerHandler) defaultMux.HandleFunc("/", fs.filerHandler)
if defaultMux != readonlyMux { if defaultMux != readonlyMux {