mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
add filer notification
This commit is contained in:
parent
75d63db60d
commit
f036ef8a3c
|
@ -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 {
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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
|
||||||
|
}
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
38
weed/filer2/filer_notify.go
Normal file
38
weed/filer2/filer_notify.go
Normal 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,
|
||||||
|
}
|
||||||
|
}
|
82
weed/msgqueue/configuration.go
Normal file
82
weed/msgqueue/configuration.go
Normal 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
|
||||||
|
}
|
76
weed/msgqueue/kafka/kafka_queue.go
Normal file
76
weed/msgqueue/kafka/kafka_queue.go
Normal 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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
29
weed/msgqueue/log/log_queue.go
Normal file
29
weed/msgqueue/log/log_queue.go
Normal 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
|
||||||
|
}
|
11
weed/msgqueue/message_queue.go
Normal file
11
weed/msgqueue/message_queue.go
Normal 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
|
||||||
|
}
|
|
@ -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;
|
||||||
|
|
|
@ -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,
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
Loading…
Reference in a new issue