Browse Source

writing meta logs is working

pull/1287/head
Chris Lu 5 years ago
parent
commit
50a5018b7f
  1. 19
      other/java/client/src/main/proto/filer.proto
  2. 2
      weed/filer2/filer.go
  3. 135
      weed/filer2/filer_notify.go
  4. 70
      weed/filer2/filer_notify_append.go
  5. 19
      weed/pb/filer.proto
  6. 384
      weed/pb/filer_pb/filer.pb.go
  7. 20
      weed/pb/master_pb/master.pb.go
  8. 1
      weed/pb/volume_server.proto
  9. 26
      weed/pb/volume_server_pb/volume_server.pb.go
  10. 70
      weed/server/filer_grpc_server_listen.go
  11. 7
      weed/server/filer_server.go
  12. 17
      weed/util/bytes.go

19
other/java/client/src/main/proto/filer.proto

@ -42,6 +42,9 @@ service SeaweedFiler {
rpc GetFilerConfiguration (GetFilerConfigurationRequest) returns (GetFilerConfigurationResponse) {
}
rpc ListenForEvents (ListenForEventsRequest) returns (stream FullEventNotification) {
}
}
//////////////////////////////////////////////////
@ -230,3 +233,19 @@ message GetFilerConfigurationResponse {
string dir_queues = 6;
bool cipher = 7;
}
message ListenForEventsRequest {
string client_name = 1;
string directory = 2;
int64 since_sec = 3;
}
message FullEventNotification {
string directory = 1;
EventNotification event_notification = 2;
}
message LogEntry {
int64 ts_ns = 1;
int32 partition_key_hash = 2;
bytes data = 3;
}

2
weed/filer2/filer.go

@ -35,6 +35,7 @@ type Filer struct {
DirQueuesPath string
buckets *FilerBuckets
Cipher bool
metaLogBuffer *LogBuffer
}
func NewFiler(masters []string, grpcDialOption grpc.DialOption, filerGrpcPort uint32) *Filer {
@ -44,6 +45,7 @@ func NewFiler(masters []string, grpcDialOption grpc.DialOption, filerGrpcPort ui
fileIdDeletionQueue: util.NewUnboundedQueue(),
GrpcDialOption: grpcDialOption,
}
f.metaLogBuffer = NewLogBuffer(time.Minute, f.logFlushFunc)
go f.loopProcessingDeletion()

135
weed/filer2/filer_notify.go

@ -1,9 +1,17 @@
package filer2
import (
"fmt"
"strings"
"sync"
"time"
"github.com/golang/protobuf/proto"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/notification"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
)
func (f *Filer) NotifyUpdateEvent(oldEntry, newEntry *Entry, deleteChunks bool) {
@ -16,24 +24,123 @@ func (f *Filer) NotifyUpdateEvent(oldEntry, newEntry *Entry, deleteChunks bool)
return
}
if notification.Queue != nil {
println("key:", key)
if strings.HasPrefix(key, "/.meta") {
return
}
newParentPath := ""
if newEntry != nil {
newParentPath, _ = newEntry.FullPath.DirAndName()
}
eventNotification := &filer_pb.EventNotification{
OldEntry: oldEntry.ToProtoEntry(),
NewEntry: newEntry.ToProtoEntry(),
DeleteChunks: deleteChunks,
NewParentPath: newParentPath,
}
if notification.Queue != nil {
glog.V(3).Infof("notifying entry update %v", key)
notification.Queue.SendMessage(key, eventNotification)
}
f.logMetaEvent(time.Now(), key, eventNotification)
}
func (f *Filer) logMetaEvent(ts time.Time, dir string, eventNotification *filer_pb.EventNotification) {
event := &filer_pb.FullEventNotification{
Directory: dir,
EventNotification: eventNotification,
}
data, err := proto.Marshal(event)
if err != nil {
glog.Errorf("failed to marshal filer_pb.FullEventNotification %+v: %v", event, err)
return
}
f.metaLogBuffer.AddToBuffer(ts, []byte(dir), data)
}
func (f *Filer) logFlushFunc(startTime, stopTime time.Time, buf []byte) {
targetFile := fmt.Sprintf("/.meta/log/%04d/%02d/%02d/%02d/%02d/%02d-%02d.log",
startTime.Year(), startTime.Month(), startTime.Day(), startTime.Hour(), startTime.Minute(),
startTime.Second(), stopTime.Second())
newParentPath := ""
if newEntry != nil {
newParentPath, _ = newEntry.FullPath.DirAndName()
}
if err := f.appendToFile(targetFile, buf); err != nil {
glog.V(0).Infof("log write failed %s: %v", targetFile, err)
}
}
type LogBuffer struct {
buf []byte
pos int
startTime time.Time
stopTime time.Time
sizeBuf []byte
flushInterval time.Duration
flushFn func(startTime, stopTime time.Time, buf []byte)
sync.Mutex
}
func NewLogBuffer(flushInterval time.Duration, flushFn func(startTime, stopTime time.Time, buf []byte)) *LogBuffer {
lb := &LogBuffer{
buf: make([]byte, 4*0124*1024),
sizeBuf: make([]byte, 4),
flushInterval: 2 * time.Second, // flushInterval,
flushFn: flushFn,
}
go lb.loopFlush()
return lb
}
func (m *LogBuffer) loopFlush() {
for {
m.Lock()
m.flush()
m.Unlock()
time.Sleep(m.flushInterval)
}
}
func (m *LogBuffer) flush() {
if m.flushFn != nil && m.pos > 0 {
m.flushFn(m.startTime, m.stopTime, m.buf[:m.pos])
m.pos = 0
}
}
notification.Queue.SendMessage(
key,
&filer_pb.EventNotification{
OldEntry: oldEntry.ToProtoEntry(),
NewEntry: newEntry.ToProtoEntry(),
DeleteChunks: deleteChunks,
NewParentPath: newParentPath,
},
)
func (m *LogBuffer) AddToBuffer(ts time.Time, key, data []byte) {
logEntry := &filer_pb.LogEntry{
TsNs: ts.UnixNano(),
PartitionKeyHash: util.HashToInt32(key),
Data: data,
}
logEntryData, _ := proto.Marshal(logEntry)
size := len(logEntryData)
m.Lock()
defer m.Unlock()
if m.pos == 0 {
m.startTime = ts
}
if m.startTime.Add(m.flushInterval).Before(ts) || len(m.buf)-m.pos < size+4 {
m.flush()
m.startTime = ts
}
m.stopTime = ts
util.Uint32toBytes(m.sizeBuf, uint32(size))
copy(m.buf[m.pos:m.pos+4], m.sizeBuf)
copy(m.buf[m.pos+4:m.pos+4+size], logEntryData)
m.pos += size + 4
}

70
weed/filer2/filer_notify_append.go

@ -0,0 +1,70 @@
package filer2
import (
"context"
"fmt"
"os"
"time"
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
)
func (f *Filer) appendToFile(targetFile string, data []byte) error {
// assign a volume location
assignRequest := &operation.VolumeAssignRequest{
Count: 1,
}
assignResult, err := operation.Assign(f.GetMaster(), f.GrpcDialOption, assignRequest)
if err != nil {
return fmt.Errorf("AssignVolume: %v", err)
}
if assignResult.Error != "" {
return fmt.Errorf("AssignVolume error: %v", assignResult.Error)
}
// upload data
targetUrl := "http://" + assignResult.Url + "/" + assignResult.Fid
uploadResult, err := operation.UploadData(targetUrl, "", false, data, false, "", nil, assignResult.Auth)
if err != nil {
return fmt.Errorf("upload data %s: %v", targetUrl, err)
}
println("uploaded to", targetUrl)
// find out existing entry
fullpath := util.FullPath(targetFile)
entry, err := f.FindEntry(context.Background(), fullpath)
var offset int64 = 0
if err == filer_pb.ErrNotFound {
entry = &Entry{
FullPath: fullpath,
Attr: Attr{
Crtime: time.Now(),
Mtime: time.Now(),
Mode: os.FileMode(0644),
Uid: OS_UID,
Gid: OS_GID,
},
}
} else {
offset = int64(TotalSize(entry.Chunks))
}
// append to existing chunks
chunk := &filer_pb.FileChunk{
FileId: assignResult.Fid,
Offset: offset,
Size: uint64(uploadResult.Size),
Mtime: time.Now().UnixNano(),
ETag: uploadResult.ETag,
IsGzipped: uploadResult.Gzip > 0,
}
entry.Chunks = append(entry.Chunks, chunk)
// update the entry
err = f.CreateEntry(context.Background(), entry, false)
return err
}

19
weed/pb/filer.proto

@ -42,6 +42,9 @@ service SeaweedFiler {
rpc GetFilerConfiguration (GetFilerConfigurationRequest) returns (GetFilerConfigurationResponse) {
}
rpc ListenForEvents (ListenForEventsRequest) returns (stream FullEventNotification) {
}
}
//////////////////////////////////////////////////
@ -230,3 +233,19 @@ message GetFilerConfigurationResponse {
string dir_queues = 6;
bool cipher = 7;
}
message ListenForEventsRequest {
string client_name = 1;
string directory = 2;
int64 since_sec = 3;
}
message FullEventNotification {
string directory = 1;
EventNotification event_notification = 2;
}
message LogEntry {
int64 ts_ns = 1;
int32 partition_key_hash = 2;
bytes data = 3;
}

384
weed/pb/filer_pb/filer.pb.go

@ -39,6 +39,9 @@ It has these top-level messages:
StatisticsResponse
GetFilerConfigurationRequest
GetFilerConfigurationResponse
ListenForEventsRequest
FullEventNotification
LogEntry
*/
package filer_pb
@ -1087,6 +1090,94 @@ func (m *GetFilerConfigurationResponse) GetCipher() bool {
return false
}
type ListenForEventsRequest struct {
ClientName string `protobuf:"bytes,1,opt,name=client_name,json=clientName" json:"client_name,omitempty"`
Directory string `protobuf:"bytes,2,opt,name=directory" json:"directory,omitempty"`
SinceSec int64 `protobuf:"varint,3,opt,name=since_sec,json=sinceSec" json:"since_sec,omitempty"`
}
func (m *ListenForEventsRequest) Reset() { *m = ListenForEventsRequest{} }
func (m *ListenForEventsRequest) String() string { return proto.CompactTextString(m) }
func (*ListenForEventsRequest) ProtoMessage() {}
func (*ListenForEventsRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{30} }
func (m *ListenForEventsRequest) GetClientName() string {
if m != nil {
return m.ClientName
}
return ""
}
func (m *ListenForEventsRequest) GetDirectory() string {
if m != nil {
return m.Directory
}
return ""
}
func (m *ListenForEventsRequest) GetSinceSec() int64 {
if m != nil {
return m.SinceSec
}
return 0
}
type FullEventNotification struct {
Directory string `protobuf:"bytes,1,opt,name=directory" json:"directory,omitempty"`
EventNotification *EventNotification `protobuf:"bytes,2,opt,name=event_notification,json=eventNotification" json:"event_notification,omitempty"`
}
func (m *FullEventNotification) Reset() { *m = FullEventNotification{} }
func (m *FullEventNotification) String() string { return proto.CompactTextString(m) }
func (*FullEventNotification) ProtoMessage() {}
func (*FullEventNotification) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{31} }
func (m *FullEventNotification) GetDirectory() string {
if m != nil {
return m.Directory
}
return ""
}
func (m *FullEventNotification) GetEventNotification() *EventNotification {
if m != nil {
return m.EventNotification
}
return nil
}
type LogEntry struct {
TsNs int64 `protobuf:"varint,1,opt,name=ts_ns,json=tsNs" json:"ts_ns,omitempty"`
PartitionKeyHash int32 `protobuf:"varint,2,opt,name=partition_key_hash,json=partitionKeyHash" json:"partition_key_hash,omitempty"`
Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
}
func (m *LogEntry) Reset() { *m = LogEntry{} }
func (m *LogEntry) String() string { return proto.CompactTextString(m) }
func (*LogEntry) ProtoMessage() {}
func (*LogEntry) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{32} }
func (m *LogEntry) GetTsNs() int64 {
if m != nil {
return m.TsNs
}
return 0
}
func (m *LogEntry) GetPartitionKeyHash() int32 {
if m != nil {
return m.PartitionKeyHash
}
return 0
}
func (m *LogEntry) GetData() []byte {
if m != nil {
return m.Data
}
return nil
}
func init() {
proto.RegisterType((*LookupDirectoryEntryRequest)(nil), "filer_pb.LookupDirectoryEntryRequest")
proto.RegisterType((*LookupDirectoryEntryResponse)(nil), "filer_pb.LookupDirectoryEntryResponse")
@ -1118,6 +1209,9 @@ func init() {
proto.RegisterType((*StatisticsResponse)(nil), "filer_pb.StatisticsResponse")
proto.RegisterType((*GetFilerConfigurationRequest)(nil), "filer_pb.GetFilerConfigurationRequest")
proto.RegisterType((*GetFilerConfigurationResponse)(nil), "filer_pb.GetFilerConfigurationResponse")
proto.RegisterType((*ListenForEventsRequest)(nil), "filer_pb.ListenForEventsRequest")
proto.RegisterType((*FullEventNotification)(nil), "filer_pb.FullEventNotification")
proto.RegisterType((*LogEntry)(nil), "filer_pb.LogEntry")
}
// Reference imports to suppress errors if they are not otherwise used.
@ -1142,6 +1236,7 @@ type SeaweedFilerClient interface {
DeleteCollection(ctx context.Context, in *DeleteCollectionRequest, opts ...grpc.CallOption) (*DeleteCollectionResponse, error)
Statistics(ctx context.Context, in *StatisticsRequest, opts ...grpc.CallOption) (*StatisticsResponse, error)
GetFilerConfiguration(ctx context.Context, in *GetFilerConfigurationRequest, opts ...grpc.CallOption) (*GetFilerConfigurationResponse, error)
ListenForEvents(ctx context.Context, in *ListenForEventsRequest, opts ...grpc.CallOption) (SeaweedFiler_ListenForEventsClient, error)
}
type seaweedFilerClient struct {
@ -1274,6 +1369,38 @@ func (c *seaweedFilerClient) GetFilerConfiguration(ctx context.Context, in *GetF
return out, nil
}
func (c *seaweedFilerClient) ListenForEvents(ctx context.Context, in *ListenForEventsRequest, opts ...grpc.CallOption) (SeaweedFiler_ListenForEventsClient, error) {
stream, err := grpc.NewClientStream(ctx, &_SeaweedFiler_serviceDesc.Streams[1], c.cc, "/filer_pb.SeaweedFiler/ListenForEvents", opts...)
if err != nil {
return nil, err
}
x := &seaweedFilerListenForEventsClient{stream}
if err := x.ClientStream.SendMsg(in); err != nil {
return nil, err
}
if err := x.ClientStream.CloseSend(); err != nil {
return nil, err
}
return x, nil
}
type SeaweedFiler_ListenForEventsClient interface {
Recv() (*FullEventNotification, error)
grpc.ClientStream
}
type seaweedFilerListenForEventsClient struct {
grpc.ClientStream
}
func (x *seaweedFilerListenForEventsClient) Recv() (*FullEventNotification, error) {
m := new(FullEventNotification)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
// Server API for SeaweedFiler service
type SeaweedFilerServer interface {
@ -1288,6 +1415,7 @@ type SeaweedFilerServer interface {
DeleteCollection(context.Context, *DeleteCollectionRequest) (*DeleteCollectionResponse, error)
Statistics(context.Context, *StatisticsRequest) (*StatisticsResponse, error)
GetFilerConfiguration(context.Context, *GetFilerConfigurationRequest) (*GetFilerConfigurationResponse, error)
ListenForEvents(*ListenForEventsRequest, SeaweedFiler_ListenForEventsServer) error
}
func RegisterSeaweedFilerServer(s *grpc.Server, srv SeaweedFilerServer) {
@ -1495,6 +1623,27 @@ func _SeaweedFiler_GetFilerConfiguration_Handler(srv interface{}, ctx context.Co
return interceptor(ctx, in, info, handler)
}
func _SeaweedFiler_ListenForEvents_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(ListenForEventsRequest)
if err := stream.RecvMsg(m); err != nil {
return err
}
return srv.(SeaweedFilerServer).ListenForEvents(m, &seaweedFilerListenForEventsServer{stream})
}
type SeaweedFiler_ListenForEventsServer interface {
Send(*FullEventNotification) error
grpc.ServerStream
}
type seaweedFilerListenForEventsServer struct {
grpc.ServerStream
}
func (x *seaweedFilerListenForEventsServer) Send(m *FullEventNotification) error {
return x.ServerStream.SendMsg(m)
}
var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
ServiceName: "filer_pb.SeaweedFiler",
HandlerType: (*SeaweedFilerServer)(nil),
@ -1546,6 +1695,11 @@ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
Handler: _SeaweedFiler_ListEntries_Handler,
ServerStreams: true,
},
{
StreamName: "ListenForEvents",
Handler: _SeaweedFiler_ListenForEvents_Handler,
ServerStreams: true,
},
},
Metadata: "filer.proto",
}
@ -1553,114 +1707,124 @@ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
func init() { proto.RegisterFile("filer.proto", fileDescriptor0) }
var fileDescriptor0 = []byte{
// 1739 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x58, 0xdd, 0x6e, 0xdb, 0xc8,
0x15, 0x0e, 0x25, 0xeb, 0x87, 0x47, 0x52, 0xd6, 0x1e, 0x3b, 0x59, 0x45, 0xb1, 0x53, 0x2f, 0xd3,
0x6c, 0x53, 0x24, 0x70, 0x03, 0x77, 0x2f, 0x76, 0xbb, 0xed, 0x45, 0xe2, 0x38, 0x8b, 0xa0, 0x49,
0x36, 0xa5, 0x93, 0xa2, 0x45, 0x81, 0x12, 0x34, 0x39, 0x92, 0xa7, 0xa6, 0x38, 0xdc, 0x99, 0x61,
0xec, 0xec, 0xa3, 0x14, 0xe8, 0x5b, 0xf4, 0xb2, 0xe8, 0x4d, 0x51, 0xb4, 0xcf, 0xd1, 0xcb, 0x5e,
0xf4, 0x19, 0x8a, 0x39, 0x43, 0x52, 0x43, 0x51, 0xb6, 0x77, 0xb1, 0xc8, 0x1d, 0xe7, 0xfc, 0xcd,
0x99, 0xef, 0xfc, 0x4a, 0x30, 0x98, 0xb2, 0x84, 0x8a, 0xbd, 0x4c, 0x70, 0xc5, 0x49, 0x1f, 0x0f,
0x41, 0x76, 0xec, 0x7d, 0x0d, 0xb7, 0x5f, 0x70, 0x7e, 0x9a, 0x67, 0x4f, 0x99, 0xa0, 0x91, 0xe2,
0xe2, 0xfd, 0x61, 0xaa, 0xc4, 0x7b, 0x9f, 0x7e, 0x93, 0x53, 0xa9, 0xc8, 0x36, 0xb8, 0x71, 0xc9,
0x18, 0x3b, 0xbb, 0xce, 0x7d, 0xd7, 0x5f, 0x10, 0x08, 0x81, 0xb5, 0x34, 0x9c, 0xd3, 0x71, 0x0b,
0x19, 0xf8, 0xed, 0x1d, 0xc2, 0xf6, 0x6a, 0x83, 0x32, 0xe3, 0xa9, 0xa4, 0xe4, 0x1e, 0x74, 0xa8,
0x26, 0xa0, 0xb5, 0xc1, 0xfe, 0x47, 0x7b, 0xa5, 0x2b, 0x7b, 0x46, 0xce, 0x70, 0xbd, 0xbf, 0x3b,
0x40, 0x5e, 0x30, 0xa9, 0x34, 0x91, 0x51, 0xf9, 0xdd, 0xfc, 0xb9, 0x09, 0xdd, 0x4c, 0xd0, 0x29,
0x3b, 0x2f, 0x3c, 0x2a, 0x4e, 0xe4, 0x21, 0x6c, 0x48, 0x15, 0x0a, 0xf5, 0x4c, 0xf0, 0xf9, 0x33,
0x96, 0xd0, 0x57, 0xda, 0xe9, 0x36, 0x8a, 0x34, 0x19, 0x64, 0x0f, 0x08, 0x4b, 0xa3, 0x24, 0x97,
0xec, 0x1d, 0x3d, 0x2a, 0xb9, 0xe3, 0xb5, 0x5d, 0xe7, 0x7e, 0xdf, 0x5f, 0xc1, 0x21, 0x5b, 0xd0,
0x49, 0xd8, 0x9c, 0xa9, 0x71, 0x67, 0xd7, 0xb9, 0x3f, 0xf2, 0xcd, 0xc1, 0xfb, 0x25, 0x6c, 0xd6,
0xfc, 0xff, 0x7e, 0xcf, 0xff, 0x4b, 0x0b, 0x3a, 0x48, 0xa8, 0x30, 0x76, 0x16, 0x18, 0x93, 0x4f,
0x60, 0xc8, 0x64, 0xb0, 0x00, 0xa2, 0x85, 0xbe, 0x0d, 0x98, 0xac, 0x30, 0x27, 0x0f, 0xa0, 0x1b,
0x9d, 0xe4, 0xe9, 0xa9, 0x1c, 0xb7, 0x77, 0xdb, 0xf7, 0x07, 0xfb, 0x9b, 0x8b, 0x8b, 0xf4, 0x43,
0x0f, 0x34, 0xcf, 0x2f, 0x44, 0xc8, 0xe7, 0x00, 0xa1, 0x52, 0x82, 0x1d, 0xe7, 0x8a, 0x4a, 0x7c,
0xe9, 0x60, 0x7f, 0x6c, 0x29, 0xe4, 0x92, 0x3e, 0xae, 0xf8, 0xbe, 0x25, 0x4b, 0xbe, 0x80, 0x3e,
0x3d, 0x57, 0x34, 0x8d, 0x69, 0x3c, 0xee, 0xe0, 0x45, 0x3b, 0x4b, 0x2f, 0xda, 0x3b, 0x2c, 0xf8,
0xe6, 0x7d, 0x95, 0xf8, 0xe4, 0x4b, 0x18, 0xd5, 0x58, 0x64, 0x1d, 0xda, 0xa7, 0xb4, 0x8c, 0xaa,
0xfe, 0xd4, 0xc8, 0xbe, 0x0b, 0x93, 0xdc, 0x24, 0xd8, 0xd0, 0x37, 0x87, 0x5f, 0xb4, 0x3e, 0x77,
0xbc, 0xa7, 0xe0, 0x3e, 0xcb, 0x93, 0xa4, 0x52, 0x8c, 0x99, 0x28, 0x15, 0x63, 0x26, 0x16, 0x28,
0xb7, 0x2e, 0x45, 0xf9, 0x6f, 0x0e, 0x6c, 0x1c, 0xbe, 0xa3, 0xa9, 0x7a, 0xc5, 0x15, 0x9b, 0xb2,
0x28, 0x54, 0x8c, 0xa7, 0xe4, 0x21, 0xb8, 0x3c, 0x89, 0x83, 0x4b, 0xc3, 0xd4, 0xe7, 0x49, 0xe1,
0xf5, 0x43, 0x70, 0x53, 0x7a, 0x16, 0x5c, 0x7a, 0x5d, 0x3f, 0xa5, 0x67, 0x46, 0xfa, 0x2e, 0x8c,
0x62, 0x9a, 0x50, 0x45, 0x83, 0x2a, 0x3a, 0x3a, 0x74, 0x43, 0x43, 0x3c, 0x30, 0xe1, 0xf8, 0x14,
0x3e, 0xd2, 0x26, 0xb3, 0x50, 0xd0, 0x54, 0x05, 0x59, 0xa8, 0x4e, 0x30, 0x26, 0xae, 0x3f, 0x4a,
0xe9, 0xd9, 0x6b, 0xa4, 0xbe, 0x0e, 0xd5, 0x89, 0xf7, 0xd7, 0x16, 0xb8, 0x55, 0x30, 0xc9, 0xc7,
0xd0, 0xd3, 0xd7, 0x06, 0x2c, 0x2e, 0x90, 0xe8, 0xea, 0xe3, 0xf3, 0x58, 0x57, 0x05, 0x9f, 0x4e,
0x25, 0x55, 0xe8, 0x5e, 0xdb, 0x2f, 0x4e, 0x3a, 0xb3, 0x24, 0xfb, 0xd6, 0x14, 0xc2, 0x9a, 0x8f,
0xdf, 0x1a, 0xf1, 0xb9, 0x62, 0x73, 0x8a, 0x17, 0xb6, 0x7d, 0x73, 0x20, 0x9b, 0xd0, 0xa1, 0x81,
0x0a, 0x67, 0x98, 0xe1, 0xae, 0xbf, 0x46, 0xdf, 0x84, 0x33, 0xf2, 0x63, 0xb8, 0x2e, 0x79, 0x2e,
0x22, 0x1a, 0x94, 0xd7, 0x76, 0x91, 0x3b, 0x34, 0xd4, 0x67, 0xe6, 0x72, 0x0f, 0xda, 0x53, 0x16,
0x8f, 0x7b, 0x08, 0xcc, 0x7a, 0x3d, 0x09, 0x9f, 0xc7, 0xbe, 0x66, 0x92, 0x9f, 0x01, 0x54, 0x96,
0xe2, 0x71, 0xff, 0x02, 0x51, 0xb7, 0xb4, 0x1b, 0x93, 0x1d, 0x80, 0x88, 0x65, 0x27, 0x54, 0x04,
0x3a, 0x61, 0x5c, 0x4c, 0x0e, 0xd7, 0x50, 0x7e, 0x4d, 0xdf, 0x6b, 0x36, 0x93, 0xc1, 0xec, 0x5b,
0x96, 0x65, 0x34, 0x1e, 0x03, 0x22, 0xec, 0x32, 0xf9, 0x95, 0x21, 0x78, 0xbf, 0x83, 0x6e, 0xe1,
0xdc, 0x6d, 0x70, 0xdf, 0xf1, 0x24, 0x9f, 0x57, 0xa0, 0x8d, 0xfc, 0xbe, 0x21, 0x3c, 0x8f, 0xc9,
0x2d, 0xc0, 0x2e, 0x89, 0x57, 0xb4, 0x10, 0x22, 0xc4, 0x57, 0x5f, 0x70, 0x13, 0xba, 0x11, 0xe7,
0xa7, 0xcc, 0x60, 0xd7, 0xf3, 0x8b, 0x93, 0xf7, 0xbf, 0x16, 0x5c, 0xaf, 0x17, 0x8b, 0xbe, 0x02,
0xad, 0x20, 0xd2, 0x0e, 0x9a, 0x41, 0xb3, 0x47, 0x35, 0xb4, 0x5b, 0x36, 0xda, 0xa5, 0xca, 0x9c,
0xc7, 0xe6, 0x82, 0x91, 0x51, 0x79, 0xc9, 0x63, 0xaa, 0x73, 0x3d, 0x67, 0x31, 0x86, 0x67, 0xe4,
0xeb, 0x4f, 0x4d, 0x99, 0xb1, 0xb8, 0x68, 0x3e, 0xfa, 0x13, 0xdd, 0x13, 0x68, 0xb7, 0x6b, 0x02,
0x6e, 0x4e, 0x3a, 0xe0, 0x73, 0x4d, 0xed, 0x99, 0x28, 0xea, 0x6f, 0xb2, 0x0b, 0x03, 0x41, 0xb3,
0xa4, 0xc8, 0x7d, 0x04, 0xdf, 0xf5, 0x6d, 0x12, 0xb9, 0x03, 0x10, 0xf1, 0x24, 0xa1, 0x11, 0x0a,
0xb8, 0x28, 0x60, 0x51, 0x74, 0xde, 0x29, 0x95, 0x04, 0x92, 0x46, 0x08, 0x75, 0xc7, 0xef, 0x2a,
0x95, 0x1c, 0xd1, 0x48, 0xbf, 0x23, 0x97, 0x54, 0x04, 0xd8, 0xbe, 0x06, 0xa8, 0xd7, 0xd7, 0x04,
0x6c, 0xb2, 0x3b, 0x00, 0x33, 0xc1, 0xf3, 0xcc, 0x70, 0x87, 0xbb, 0x6d, 0xdd, 0xc9, 0x91, 0x82,
0xec, 0x7b, 0x70, 0x5d, 0xbe, 0x9f, 0x27, 0x2c, 0x3d, 0x0d, 0x54, 0x28, 0x66, 0x54, 0x8d, 0x47,
0xa6, 0x02, 0x0a, 0xea, 0x1b, 0x24, 0x7a, 0x19, 0x90, 0x03, 0x41, 0x43, 0x45, 0xbf, 0xc7, 0xd0,
0xfa, 0x6e, 0xbd, 0x81, 0xdc, 0x80, 0x2e, 0x0f, 0xe8, 0x79, 0x94, 0x14, 0x25, 0xda, 0xe1, 0x87,
0xe7, 0x51, 0xe2, 0x3d, 0x80, 0xcd, 0xda, 0x8d, 0x45, 0x5b, 0xdf, 0x82, 0x0e, 0x15, 0x82, 0x97,
0x4d, 0xc8, 0x1c, 0xbc, 0xdf, 0x03, 0x79, 0x9b, 0xc5, 0x1f, 0xc2, 0x3d, 0xef, 0x06, 0x6c, 0xd6,
0x4c, 0x1b, 0x3f, 0xbc, 0x7f, 0x3a, 0x40, 0x9e, 0x62, 0x2f, 0xf9, 0x61, 0x63, 0x5c, 0x57, 0xb7,
0x1e, 0x31, 0xa6, 0x57, 0xc5, 0xa1, 0x0a, 0x8b, 0x01, 0x38, 0x64, 0xd2, 0xd8, 0x7f, 0x1a, 0xaa,
0xb0, 0x18, 0x44, 0x82, 0x46, 0xb9, 0xd0, 0x33, 0x11, 0x93, 0x10, 0x07, 0x91, 0x5f, 0x92, 0xc8,
0x67, 0x70, 0x93, 0xcd, 0x52, 0x2e, 0xe8, 0x42, 0x2c, 0x30, 0x50, 0x75, 0x51, 0x78, 0xcb, 0x70,
0x2b, 0x85, 0x43, 0x44, 0xee, 0x01, 0x6c, 0xd6, 0x9e, 0x71, 0x29, 0xcc, 0x7f, 0x76, 0x60, 0xfc,
0x58, 0xf1, 0x39, 0x8b, 0x7c, 0xaa, 0x9d, 0xaf, 0x3d, 0xfd, 0x2e, 0x8c, 0x74, 0x37, 0x5f, 0x7e,
0xfe, 0x90, 0x27, 0xf1, 0x62, 0x5a, 0xde, 0x02, 0xdd, 0xd0, 0x03, 0x0b, 0x85, 0x1e, 0x4f, 0x62,
0xcc, 0xc4, 0xbb, 0xa0, 0xbb, 0xae, 0xa5, 0x6f, 0xf6, 0x86, 0x61, 0x4a, 0xcf, 0x6a, 0xfa, 0x5a,
0x08, 0xf5, 0x4d, 0xab, 0xee, 0xa5, 0xf4, 0x4c, 0xeb, 0x7b, 0xb7, 0xe1, 0xd6, 0x0a, 0xdf, 0x8a,
0x70, 0xfd, 0xdb, 0x81, 0xcd, 0xc7, 0x52, 0xb2, 0x59, 0xfa, 0x5b, 0x6c, 0x3b, 0xa5, 0xd3, 0x5b,
0xd0, 0x89, 0x78, 0x9e, 0x2a, 0x74, 0xb6, 0xe3, 0x9b, 0xc3, 0x52, 0x25, 0xb6, 0x1a, 0x95, 0xb8,
0x54, 0xcb, 0xed, 0x66, 0x2d, 0x5b, 0xb5, 0xba, 0x56, 0xab, 0xd5, 0x1f, 0xc1, 0x40, 0x07, 0x39,
0x88, 0x68, 0xaa, 0xa8, 0x28, 0xfa, 0x3c, 0x68, 0xd2, 0x01, 0x52, 0xb4, 0x80, 0x3d, 0x8f, 0x4c,
0xab, 0x87, 0x6c, 0x31, 0x8c, 0xfe, 0xe3, 0xc0, 0x56, 0xfd, 0x29, 0x45, 0xcc, 0x2e, 0x9c, 0x4b,
0xba, 0x95, 0x89, 0xa4, 0x78, 0x87, 0xfe, 0xd4, 0x4d, 0x21, 0xcb, 0x8f, 0x13, 0x16, 0x05, 0x9a,
0x61, 0xfc, 0x77, 0x0d, 0xe5, 0xad, 0x48, 0x16, 0xa8, 0xac, 0xd9, 0xa8, 0x10, 0x58, 0x0b, 0x73,
0x75, 0x52, 0xce, 0x26, 0xfd, 0xbd, 0x84, 0x54, 0xf7, 0x2a, 0xa4, 0x7a, 0x4d, 0xa4, 0xaa, 0x4c,
0xeb, 0xdb, 0x99, 0xf6, 0x19, 0x6c, 0x9a, 0xe5, 0xb6, 0x1e, 0xae, 0x1d, 0x80, 0x6a, 0x8e, 0xc8,
0xb1, 0x63, 0x9a, 0x59, 0x39, 0x48, 0xa4, 0xf7, 0x2b, 0x70, 0x5f, 0x70, 0x63, 0x57, 0x92, 0x47,
0xe0, 0x26, 0xe5, 0x01, 0x45, 0x07, 0xfb, 0x64, 0x51, 0xe3, 0xa5, 0x9c, 0xbf, 0x10, 0xf2, 0xbe,
0x84, 0x7e, 0x49, 0x2e, 0x31, 0x73, 0x2e, 0xc2, 0xac, 0xb5, 0x84, 0x99, 0xf7, 0x0f, 0x07, 0xb6,
0xea, 0x2e, 0x17, 0x61, 0x79, 0x0b, 0xa3, 0xea, 0x8a, 0x60, 0x1e, 0x66, 0x85, 0x2f, 0x8f, 0x6c,
0x5f, 0x9a, 0x6a, 0x95, 0x83, 0xf2, 0x65, 0x98, 0x99, 0x5c, 0x1e, 0x26, 0x16, 0x69, 0xf2, 0x06,
0x36, 0x1a, 0x22, 0x2b, 0x36, 0xbb, 0x9f, 0xda, 0x9b, 0x5d, 0x6d, 0x3b, 0xad, 0xb4, 0xed, 0x75,
0xef, 0x0b, 0xf8, 0xd8, 0xb4, 0x83, 0x83, 0x2a, 0x86, 0x25, 0xf6, 0xf5, 0x50, 0x3b, 0xcb, 0xa1,
0xf6, 0x26, 0x30, 0x6e, 0xaa, 0x16, 0xe5, 0x37, 0x83, 0x8d, 0x23, 0x15, 0x2a, 0x26, 0x15, 0x8b,
0xaa, 0x9f, 0x18, 0x4b, 0xb9, 0xe1, 0x5c, 0x35, 0x11, 0x9b, 0x75, 0xb8, 0x0e, 0x6d, 0xa5, 0xca,
0xfc, 0xd5, 0x9f, 0x3a, 0x0a, 0xc4, 0xbe, 0xa9, 0x88, 0xc1, 0x07, 0xb8, 0x4a, 0xe7, 0x83, 0xe2,
0x2a, 0x4c, 0xcc, 0xc6, 0xb1, 0x86, 0x1b, 0x87, 0x8b, 0x14, 0x5c, 0x39, 0xcc, 0x50, 0x8e, 0x0d,
0xb7, 0x63, 0xf6, 0x11, 0x4d, 0x40, 0xe6, 0x0e, 0x00, 0x96, 0xaa, 0xa9, 0xb2, 0xae, 0xd1, 0xd5,
0x94, 0x03, 0x4d, 0xf0, 0xee, 0xc0, 0xf6, 0x57, 0x54, 0xe9, 0xdd, 0x49, 0x1c, 0xf0, 0x74, 0xca,
0x66, 0xb9, 0x08, 0xad, 0x50, 0x78, 0xff, 0x75, 0x60, 0xe7, 0x02, 0x81, 0xe2, 0xc1, 0x63, 0xe8,
0xcd, 0x43, 0xa9, 0xa8, 0x28, 0xab, 0xa4, 0x3c, 0x2e, 0x43, 0xd1, 0xba, 0x0a, 0x8a, 0x76, 0x03,
0x8a, 0x1b, 0xd0, 0x9d, 0x87, 0xe7, 0xc1, 0xfc, 0xb8, 0x58, 0x8e, 0x3a, 0xf3, 0xf0, 0xfc, 0xe5,
0x31, 0x76, 0x36, 0x26, 0x82, 0xe3, 0x3c, 0x3a, 0xa5, 0x4a, 0x56, 0x9d, 0x8d, 0x89, 0x27, 0x86,
0xa2, 0x1f, 0xad, 0x05, 0xbe, 0xc9, 0x69, 0x4e, 0x65, 0xd1, 0x2b, 0xf4, 0x70, 0xfc, 0x0d, 0x12,
0x70, 0x99, 0xc2, 0xcd, 0x12, 0xbb, 0x44, 0xdf, 0x2f, 0x4e, 0xfb, 0xff, 0xea, 0xc1, 0xf0, 0x88,
0x86, 0x67, 0x94, 0xc6, 0xf8, 0x60, 0x32, 0x2b, 0x0b, 0xad, 0xfe, 0xc3, 0x97, 0xdc, 0x5b, 0xae,
0xa8, 0x95, 0xbf, 0xb4, 0x27, 0x9f, 0x5e, 0x25, 0x56, 0xe4, 0xec, 0x35, 0xf2, 0x0a, 0x06, 0xd6,
0x2f, 0x4b, 0xb2, 0x6d, 0x29, 0x36, 0x7e, 0x30, 0x4f, 0x76, 0x2e, 0xe0, 0x96, 0xd6, 0x1e, 0x39,
0xe4, 0x05, 0x0c, 0xac, 0x95, 0xc6, 0xb6, 0xd7, 0xdc, 0xad, 0x6c, 0x7b, 0x2b, 0xf6, 0x20, 0xef,
0x9a, 0xb6, 0x66, 0x2d, 0x26, 0xb6, 0xb5, 0xe6, 0x2a, 0x64, 0x5b, 0x5b, 0xb5, 0xcd, 0xa0, 0x35,
0x6b, 0x0f, 0xb0, 0xad, 0x35, 0xb7, 0x1c, 0xdb, 0xda, 0x8a, 0xe5, 0xc1, 0xbb, 0x46, 0xfe, 0x08,
0x1b, 0x8d, 0x59, 0x4c, 0xbc, 0x85, 0xd6, 0x45, 0x4b, 0xc4, 0xe4, 0xee, 0xa5, 0x32, 0x95, 0xfd,
0xaf, 0x61, 0x68, 0x8f, 0x40, 0x62, 0x39, 0xb4, 0x62, 0xca, 0x4f, 0xee, 0x5c, 0xc4, 0xb6, 0x0d,
0xda, 0x5d, 0xd8, 0x36, 0xb8, 0x62, 0x0e, 0xd9, 0x06, 0x57, 0x35, 0x6f, 0xef, 0x1a, 0xf9, 0x03,
0xac, 0x2f, 0x77, 0x43, 0xf2, 0xc9, 0x32, 0x6c, 0x8d, 0x26, 0x3b, 0xf1, 0x2e, 0x13, 0xa9, 0x8c,
0x3f, 0x07, 0x58, 0x34, 0x39, 0x72, 0x7b, 0xa1, 0xd3, 0x68, 0xb2, 0x93, 0xed, 0xd5, 0xcc, 0xca,
0xd4, 0x9f, 0xe0, 0xc6, 0xca, 0x4e, 0x42, 0xac, 0x32, 0xb9, 0xac, 0x17, 0x4d, 0x7e, 0x72, 0xa5,
0x5c, 0x79, 0xd7, 0x93, 0x3b, 0xb0, 0x2e, 0x4d, 0x21, 0x4f, 0xe5, 0x5e, 0x94, 0x30, 0x9a, 0xaa,
0x27, 0x80, 0x1a, 0xaf, 0x05, 0x57, 0xfc, 0xb8, 0x8b, 0xff, 0x99, 0xfd, 0xfc, 0xff, 0x01, 0x00,
0x00, 0xff, 0xff, 0x84, 0x9e, 0xb9, 0x33, 0x42, 0x13, 0x00, 0x00,
// 1901 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x58, 0x5f, 0x6f, 0xdc, 0xc6,
0x11, 0x37, 0xef, 0x9f, 0x8e, 0x73, 0x77, 0xb6, 0xb4, 0x27, 0x39, 0xe7, 0x93, 0x64, 0x2b, 0x74,
0x9d, 0xba, 0xb0, 0xa1, 0x1a, 0x6a, 0x1e, 0x92, 0xa6, 0x7d, 0xb0, 0x65, 0x29, 0x75, 0x63, 0x2b,
0x2e, 0x65, 0x17, 0x29, 0x0a, 0x94, 0xa0, 0xc8, 0xd5, 0x69, 0x2b, 0x1e, 0xc9, 0xec, 0x2e, 0xf5,
0x27, 0x6f, 0xfd, 0x1a, 0x05, 0xfa, 0xd0, 0xef, 0xd0, 0xc7, 0xa2, 0x2f, 0x45, 0x81, 0x7e, 0x8e,
0x3e, 0xf6, 0xa1, 0x9f, 0xa1, 0xd8, 0x59, 0x92, 0xb7, 0x3c, 0x9e, 0xa4, 0x04, 0x41, 0xde, 0x76,
0x67, 0x66, 0x67, 0x67, 0xe7, 0xcf, 0x6f, 0x86, 0x84, 0xde, 0x31, 0x8b, 0x28, 0xdf, 0x4e, 0x79,
0x22, 0x13, 0xd2, 0xc5, 0x8d, 0x97, 0x1e, 0x39, 0x5f, 0xc2, 0xfa, 0xeb, 0x24, 0x39, 0xcd, 0xd2,
0x97, 0x8c, 0xd3, 0x40, 0x26, 0xfc, 0x72, 0x2f, 0x96, 0xfc, 0xd2, 0xa5, 0x5f, 0x67, 0x54, 0x48,
0xb2, 0x01, 0x76, 0x58, 0x30, 0x46, 0xd6, 0x96, 0xf5, 0xd8, 0x76, 0x67, 0x04, 0x42, 0xa0, 0x15,
0xfb, 0x53, 0x3a, 0x6a, 0x20, 0x03, 0xd7, 0xce, 0x1e, 0x6c, 0x2c, 0x56, 0x28, 0xd2, 0x24, 0x16,
0x94, 0x3c, 0x82, 0x36, 0x55, 0x04, 0xd4, 0xd6, 0xdb, 0xb9, 0xb3, 0x5d, 0x98, 0xb2, 0xad, 0xe5,
0x34, 0xd7, 0xf9, 0x87, 0x05, 0xe4, 0x35, 0x13, 0x52, 0x11, 0x19, 0x15, 0xdf, 0xce, 0x9e, 0xbb,
0xd0, 0x49, 0x39, 0x3d, 0x66, 0x17, 0xb9, 0x45, 0xf9, 0x8e, 0x3c, 0x85, 0x15, 0x21, 0x7d, 0x2e,
0xf7, 0x79, 0x32, 0xdd, 0x67, 0x11, 0x3d, 0x50, 0x46, 0x37, 0x51, 0xa4, 0xce, 0x20, 0xdb, 0x40,
0x58, 0x1c, 0x44, 0x99, 0x60, 0x67, 0xf4, 0xb0, 0xe0, 0x8e, 0x5a, 0x5b, 0xd6, 0xe3, 0xae, 0xbb,
0x80, 0x43, 0x56, 0xa1, 0x1d, 0xb1, 0x29, 0x93, 0xa3, 0xf6, 0x96, 0xf5, 0x78, 0xe0, 0xea, 0x8d,
0xf3, 0x0b, 0x18, 0x56, 0xec, 0xff, 0x6e, 0xcf, 0xff, 0x4b, 0x03, 0xda, 0x48, 0x28, 0x7d, 0x6c,
0xcd, 0x7c, 0x4c, 0x3e, 0x84, 0x3e, 0x13, 0xde, 0xcc, 0x11, 0x0d, 0xb4, 0xad, 0xc7, 0x44, 0xe9,
0x73, 0xf2, 0x04, 0x3a, 0xc1, 0x49, 0x16, 0x9f, 0x8a, 0x51, 0x73, 0xab, 0xf9, 0xb8, 0xb7, 0x33,
0x9c, 0x5d, 0xa4, 0x1e, 0xba, 0xab, 0x78, 0x6e, 0x2e, 0x42, 0x3e, 0x01, 0xf0, 0xa5, 0xe4, 0xec,
0x28, 0x93, 0x54, 0xe0, 0x4b, 0x7b, 0x3b, 0x23, 0xe3, 0x40, 0x26, 0xe8, 0xf3, 0x92, 0xef, 0x1a,
0xb2, 0xe4, 0x53, 0xe8, 0xd2, 0x0b, 0x49, 0xe3, 0x90, 0x86, 0xa3, 0x36, 0x5e, 0xb4, 0x39, 0xf7,
0xa2, 0xed, 0xbd, 0x9c, 0xaf, 0xdf, 0x57, 0x8a, 0x8f, 0x3f, 0x83, 0x41, 0x85, 0x45, 0x96, 0xa1,
0x79, 0x4a, 0x8b, 0xa8, 0xaa, 0xa5, 0xf2, 0xec, 0x99, 0x1f, 0x65, 0x3a, 0xc1, 0xfa, 0xae, 0xde,
0xfc, 0xbc, 0xf1, 0x89, 0xe5, 0xbc, 0x04, 0x7b, 0x3f, 0x8b, 0xa2, 0xf2, 0x60, 0xc8, 0x78, 0x71,
0x30, 0x64, 0x7c, 0xe6, 0xe5, 0xc6, 0xb5, 0x5e, 0xfe, 0xbb, 0x05, 0x2b, 0x7b, 0x67, 0x34, 0x96,
0x07, 0x89, 0x64, 0xc7, 0x2c, 0xf0, 0x25, 0x4b, 0x62, 0xf2, 0x14, 0xec, 0x24, 0x0a, 0xbd, 0x6b,
0xc3, 0xd4, 0x4d, 0xa2, 0xdc, 0xea, 0xa7, 0x60, 0xc7, 0xf4, 0xdc, 0xbb, 0xf6, 0xba, 0x6e, 0x4c,
0xcf, 0xb5, 0xf4, 0x43, 0x18, 0x84, 0x34, 0xa2, 0x92, 0x7a, 0x65, 0x74, 0x54, 0xe8, 0xfa, 0x9a,
0xb8, 0xab, 0xc3, 0xf1, 0x11, 0xdc, 0x51, 0x2a, 0x53, 0x9f, 0xd3, 0x58, 0x7a, 0xa9, 0x2f, 0x4f,
0x30, 0x26, 0xb6, 0x3b, 0x88, 0xe9, 0xf9, 0x5b, 0xa4, 0xbe, 0xf5, 0xe5, 0x89, 0xf3, 0xb7, 0x06,
0xd8, 0x65, 0x30, 0xc9, 0x07, 0xb0, 0xa4, 0xae, 0xf5, 0x58, 0x98, 0x7b, 0xa2, 0xa3, 0xb6, 0xaf,
0x42, 0x55, 0x15, 0xc9, 0xf1, 0xb1, 0xa0, 0x12, 0xcd, 0x6b, 0xba, 0xf9, 0x4e, 0x65, 0x96, 0x60,
0xdf, 0xe8, 0x42, 0x68, 0xb9, 0xb8, 0x56, 0x1e, 0x9f, 0x4a, 0x36, 0xa5, 0x78, 0x61, 0xd3, 0xd5,
0x1b, 0x32, 0x84, 0x36, 0xf5, 0xa4, 0x3f, 0xc1, 0x0c, 0xb7, 0xdd, 0x16, 0x7d, 0xe7, 0x4f, 0xc8,
0x8f, 0xe0, 0xb6, 0x48, 0x32, 0x1e, 0x50, 0xaf, 0xb8, 0xb6, 0x83, 0xdc, 0xbe, 0xa6, 0xee, 0xeb,
0xcb, 0x1d, 0x68, 0x1e, 0xb3, 0x70, 0xb4, 0x84, 0x8e, 0x59, 0xae, 0x26, 0xe1, 0xab, 0xd0, 0x55,
0x4c, 0xf2, 0x53, 0x80, 0x52, 0x53, 0x38, 0xea, 0x5e, 0x21, 0x6a, 0x17, 0x7a, 0x43, 0xb2, 0x09,
0x10, 0xb0, 0xf4, 0x84, 0x72, 0x4f, 0x25, 0x8c, 0x8d, 0xc9, 0x61, 0x6b, 0xca, 0x17, 0xf4, 0x52,
0xb1, 0x99, 0xf0, 0x26, 0xdf, 0xb0, 0x34, 0xa5, 0xe1, 0x08, 0xd0, 0xc3, 0x36, 0x13, 0x9f, 0x6b,
0x82, 0xf3, 0x15, 0x74, 0x72, 0xe3, 0xd6, 0xc1, 0x3e, 0x4b, 0xa2, 0x6c, 0x5a, 0x3a, 0x6d, 0xe0,
0x76, 0x35, 0xe1, 0x55, 0x48, 0xee, 0x01, 0xa2, 0x24, 0x5e, 0xd1, 0x40, 0x17, 0xa1, 0x7f, 0xd5,
0x05, 0x77, 0xa1, 0x13, 0x24, 0xc9, 0x29, 0xd3, 0xbe, 0x5b, 0x72, 0xf3, 0x9d, 0xf3, 0xbf, 0x06,
0xdc, 0xae, 0x16, 0x8b, 0xba, 0x02, 0xb5, 0xa0, 0xa7, 0x2d, 0x54, 0x83, 0x6a, 0x0f, 0x2b, 0xde,
0x6e, 0x98, 0xde, 0x2e, 0x8e, 0x4c, 0x93, 0x50, 0x5f, 0x30, 0xd0, 0x47, 0xde, 0x24, 0x21, 0x55,
0xb9, 0x9e, 0xb1, 0x10, 0xc3, 0x33, 0x70, 0xd5, 0x52, 0x51, 0x26, 0x2c, 0xcc, 0xc1, 0x47, 0x2d,
0xd1, 0x3c, 0x8e, 0x7a, 0x3b, 0x3a, 0xe0, 0x7a, 0xa7, 0x02, 0x3e, 0x55, 0xd4, 0x25, 0x1d, 0x45,
0xb5, 0x26, 0x5b, 0xd0, 0xe3, 0x34, 0x8d, 0xf2, 0xdc, 0x47, 0xe7, 0xdb, 0xae, 0x49, 0x22, 0xf7,
0x01, 0x82, 0x24, 0x8a, 0x68, 0x80, 0x02, 0x36, 0x0a, 0x18, 0x14, 0x95, 0x77, 0x52, 0x46, 0x9e,
0xa0, 0x01, 0xba, 0xba, 0xed, 0x76, 0xa4, 0x8c, 0x0e, 0x69, 0xa0, 0xde, 0x91, 0x09, 0xca, 0x3d,
0x84, 0xaf, 0x1e, 0x9e, 0xeb, 0x2a, 0x02, 0x82, 0xec, 0x26, 0xc0, 0x84, 0x27, 0x59, 0xaa, 0xb9,
0xfd, 0xad, 0xa6, 0x42, 0x72, 0xa4, 0x20, 0xfb, 0x11, 0xdc, 0x16, 0x97, 0xd3, 0x88, 0xc5, 0xa7,
0x9e, 0xf4, 0xf9, 0x84, 0xca, 0xd1, 0x40, 0x57, 0x40, 0x4e, 0x7d, 0x87, 0x44, 0x27, 0x05, 0xb2,
0xcb, 0xa9, 0x2f, 0xe9, 0x77, 0x68, 0x5a, 0xdf, 0x0e, 0x1b, 0xc8, 0x1a, 0x74, 0x12, 0x8f, 0x5e,
0x04, 0x51, 0x5e, 0xa2, 0xed, 0x64, 0xef, 0x22, 0x88, 0x9c, 0x27, 0x30, 0xac, 0xdc, 0x98, 0xc3,
0xfa, 0x2a, 0xb4, 0x29, 0xe7, 0x49, 0x01, 0x42, 0x7a, 0xe3, 0xfc, 0x0e, 0xc8, 0xfb, 0x34, 0xfc,
0x21, 0xcc, 0x73, 0xd6, 0x60, 0x58, 0x51, 0xad, 0xed, 0x70, 0xfe, 0x65, 0x01, 0x79, 0x89, 0x58,
0xf2, 0xfd, 0xda, 0xb8, 0xaa, 0x6e, 0xd5, 0x62, 0x34, 0x56, 0x85, 0xbe, 0xf4, 0xf3, 0x06, 0xd8,
0x67, 0x42, 0xeb, 0x7f, 0xe9, 0x4b, 0x3f, 0x6f, 0x44, 0x9c, 0x06, 0x19, 0x57, 0x3d, 0x11, 0x93,
0x10, 0x1b, 0x91, 0x5b, 0x90, 0xc8, 0xc7, 0x70, 0x97, 0x4d, 0xe2, 0x84, 0xd3, 0x99, 0x98, 0xa7,
0x5d, 0xd5, 0x41, 0xe1, 0x55, 0xcd, 0x2d, 0x0f, 0xec, 0xa1, 0xe7, 0x9e, 0xc0, 0xb0, 0xf2, 0x8c,
0x6b, 0xdd, 0xfc, 0x67, 0x0b, 0x46, 0xcf, 0x65, 0x32, 0x65, 0x81, 0x4b, 0x95, 0xf1, 0x95, 0xa7,
0x3f, 0x84, 0x81, 0x42, 0xf3, 0xf9, 0xe7, 0xf7, 0x93, 0x28, 0x9c, 0x75, 0xcb, 0x7b, 0xa0, 0x00,
0xdd, 0x33, 0xbc, 0xb0, 0x94, 0x44, 0x21, 0x66, 0xe2, 0x43, 0x50, 0xa8, 0x6b, 0x9c, 0xd7, 0x73,
0x43, 0x3f, 0xa6, 0xe7, 0x95, 0xf3, 0x4a, 0x08, 0xcf, 0x6b, 0xa8, 0x5e, 0x8a, 0xe9, 0xb9, 0x3a,
0xef, 0xac, 0xc3, 0xbd, 0x05, 0xb6, 0xe5, 0xe1, 0xfa, 0xb7, 0x05, 0xc3, 0xe7, 0x42, 0xb0, 0x49,
0xfc, 0x5b, 0x84, 0x9d, 0xc2, 0xe8, 0x55, 0x68, 0x07, 0x49, 0x16, 0x4b, 0x34, 0xb6, 0xed, 0xea,
0xcd, 0x5c, 0x25, 0x36, 0x6a, 0x95, 0x38, 0x57, 0xcb, 0xcd, 0x7a, 0x2d, 0x1b, 0xb5, 0xda, 0xaa,
0xd4, 0xea, 0x03, 0xe8, 0xa9, 0x20, 0x7b, 0x01, 0x8d, 0x25, 0xe5, 0x39, 0xce, 0x83, 0x22, 0xed,
0x22, 0x45, 0x09, 0x98, 0xfd, 0x48, 0x43, 0x3d, 0xa4, 0xb3, 0x66, 0xf4, 0x1f, 0x0b, 0x56, 0xab,
0x4f, 0xc9, 0x63, 0x76, 0x65, 0x5f, 0x52, 0x50, 0xc6, 0xa3, 0xfc, 0x1d, 0x6a, 0xa9, 0x40, 0x21,
0xcd, 0x8e, 0x22, 0x16, 0x78, 0x8a, 0xa1, 0xed, 0xb7, 0x35, 0xe5, 0x3d, 0x8f, 0x66, 0x5e, 0x69,
0x99, 0x5e, 0x21, 0xd0, 0xf2, 0x33, 0x79, 0x52, 0xf4, 0x26, 0xb5, 0x9e, 0xf3, 0x54, 0xe7, 0x26,
0x4f, 0x2d, 0xd5, 0x3d, 0x55, 0x66, 0x5a, 0xd7, 0xcc, 0xb4, 0x8f, 0x61, 0xa8, 0x87, 0xdb, 0x6a,
0xb8, 0x36, 0x01, 0xca, 0x3e, 0x22, 0x46, 0x96, 0x06, 0xb3, 0xa2, 0x91, 0x08, 0xe7, 0x97, 0x60,
0xbf, 0x4e, 0xb4, 0x5e, 0x41, 0x9e, 0x81, 0x1d, 0x15, 0x1b, 0x14, 0xed, 0xed, 0x90, 0x59, 0x8d,
0x17, 0x72, 0xee, 0x4c, 0xc8, 0xf9, 0x0c, 0xba, 0x05, 0xb9, 0xf0, 0x99, 0x75, 0x95, 0xcf, 0x1a,
0x73, 0x3e, 0x73, 0xfe, 0x69, 0xc1, 0x6a, 0xd5, 0xe4, 0x3c, 0x2c, 0xef, 0x61, 0x50, 0x5e, 0xe1,
0x4d, 0xfd, 0x34, 0xb7, 0xe5, 0x99, 0x69, 0x4b, 0xfd, 0x58, 0x69, 0xa0, 0x78, 0xe3, 0xa7, 0x3a,
0x97, 0xfb, 0x91, 0x41, 0x1a, 0xbf, 0x83, 0x95, 0x9a, 0xc8, 0x82, 0xc9, 0xee, 0x27, 0xe6, 0x64,
0x57, 0x99, 0x4e, 0xcb, 0xd3, 0xe6, 0xb8, 0xf7, 0x29, 0x7c, 0xa0, 0xe1, 0x60, 0xb7, 0x8c, 0x61,
0xe1, 0xfb, 0x6a, 0xa8, 0xad, 0xf9, 0x50, 0x3b, 0x63, 0x18, 0xd5, 0x8f, 0xe6, 0xe5, 0x37, 0x81,
0x95, 0x43, 0xe9, 0x4b, 0x26, 0x24, 0x0b, 0xca, 0x4f, 0x8c, 0xb9, 0xdc, 0xb0, 0x6e, 0xea, 0x88,
0xf5, 0x3a, 0x5c, 0x86, 0xa6, 0x94, 0x45, 0xfe, 0xaa, 0xa5, 0x8a, 0x02, 0x31, 0x6f, 0xca, 0x63,
0xf0, 0x03, 0x5c, 0xa5, 0xf2, 0x41, 0x26, 0xd2, 0x8f, 0xf4, 0xc4, 0xd1, 0xc2, 0x89, 0xc3, 0x46,
0x0a, 0x8e, 0x1c, 0xba, 0x29, 0x87, 0x9a, 0xdb, 0xd6, 0xf3, 0x88, 0x22, 0x20, 0x73, 0x13, 0x00,
0x4b, 0x55, 0x57, 0x59, 0x47, 0x9f, 0x55, 0x94, 0x5d, 0x45, 0x70, 0xee, 0xc3, 0xc6, 0xe7, 0x54,
0xaa, 0xd9, 0x89, 0xef, 0x26, 0xf1, 0x31, 0x9b, 0x64, 0xdc, 0x37, 0x42, 0xe1, 0xfc, 0xd7, 0x82,
0xcd, 0x2b, 0x04, 0xf2, 0x07, 0x8f, 0x60, 0x69, 0xea, 0x0b, 0x49, 0x79, 0x51, 0x25, 0xc5, 0x76,
0xde, 0x15, 0x8d, 0x9b, 0x5c, 0xd1, 0xac, 0xb9, 0x62, 0x0d, 0x3a, 0x53, 0xff, 0xc2, 0x9b, 0x1e,
0xe5, 0xc3, 0x51, 0x7b, 0xea, 0x5f, 0xbc, 0x39, 0x42, 0x64, 0x63, 0xdc, 0x3b, 0xca, 0x82, 0x53,
0x2a, 0x45, 0x89, 0x6c, 0x8c, 0xbf, 0xd0, 0x14, 0xf5, 0x68, 0x25, 0xf0, 0x75, 0x46, 0x33, 0x2a,
0x72, 0xac, 0x50, 0xcd, 0xf1, 0x37, 0x48, 0xc0, 0x61, 0x0a, 0x27, 0x4b, 0x44, 0x89, 0xae, 0x9b,
0xef, 0x1c, 0x09, 0x77, 0xd5, 0xf7, 0x1d, 0x8d, 0xf7, 0x13, 0x8e, 0xdf, 0x10, 0x65, 0x02, 0x3d,
0x80, 0x5e, 0x10, 0x31, 0x05, 0x95, 0xc6, 0x87, 0x1b, 0x68, 0x12, 0xb6, 0x94, 0x4a, 0x37, 0x6e,
0xcc, 0x77, 0xe3, 0x75, 0xb0, 0x05, 0x8b, 0x03, 0x8a, 0x28, 0xdd, 0xc4, 0x01, 0xae, 0x8b, 0x84,
0x43, 0x1a, 0x38, 0x7f, 0xb2, 0x60, 0x0d, 0x3f, 0x7c, 0x6a, 0x5f, 0x2d, 0xd7, 0xb7, 0xf8, 0x5f,
0x03, 0xa1, 0x67, 0x68, 0x92, 0x71, 0x26, 0x2f, 0xbe, 0x75, 0x63, 0xc4, 0x98, 0x57, 0xeb, 0xae,
0xd0, 0x79, 0x92, 0xe3, 0x2b, 0x3c, 0x9a, 0xe8, 0xca, 0x1e, 0x42, 0x5b, 0x0a, 0x0f, 0x91, 0x4c,
0x19, 0xda, 0x92, 0xe2, 0x40, 0x90, 0xa7, 0x40, 0x52, 0x9f, 0x4b, 0xa6, 0xa4, 0xd5, 0xf8, 0xec,
0x9d, 0xf8, 0xe2, 0x04, 0x2f, 0x6b, 0xbb, 0xcb, 0x25, 0xe7, 0x0b, 0x7a, 0xf9, 0x2b, 0x5f, 0x9c,
0x28, 0xfc, 0xc6, 0xf9, 0xa2, 0x89, 0x63, 0x3c, 0xae, 0x77, 0xfe, 0xda, 0x85, 0xfe, 0x21, 0xf5,
0xcf, 0x29, 0x0d, 0x31, 0x9b, 0xc8, 0xa4, 0x40, 0xb1, 0xea, 0x5f, 0x05, 0xf2, 0x68, 0x1e, 0xae,
0x16, 0xfe, 0xc6, 0x18, 0x7f, 0x74, 0x93, 0x58, 0x0e, 0x08, 0xb7, 0xc8, 0x01, 0xf4, 0x8c, 0xcf,
0x76, 0xb2, 0x61, 0x1c, 0xac, 0xfd, 0x8d, 0x18, 0x6f, 0x5e, 0xc1, 0x2d, 0xb4, 0x3d, 0xb3, 0xc8,
0x6b, 0xe8, 0x19, 0xf3, 0xa2, 0xa9, 0xaf, 0x3e, 0xb8, 0x9a, 0xfa, 0x16, 0x0c, 0x99, 0xce, 0x2d,
0xa5, 0xcd, 0x98, 0xfa, 0x4c, 0x6d, 0xf5, 0x39, 0xd3, 0xd4, 0xb6, 0x68, 0x54, 0x44, 0x6d, 0xc6,
0x90, 0x65, 0x6a, 0xab, 0x8f, 0x90, 0xa6, 0xb6, 0x05, 0x93, 0x99, 0x73, 0x8b, 0xfc, 0x01, 0x56,
0x6a, 0x83, 0x0e, 0x71, 0x66, 0xa7, 0xae, 0x9a, 0xd0, 0xc6, 0x0f, 0xaf, 0x95, 0x29, 0xf5, 0x7f,
0x09, 0x7d, 0x73, 0xbe, 0x20, 0x86, 0x41, 0x0b, 0x46, 0xa8, 0xf1, 0xfd, 0xab, 0xd8, 0xa6, 0x42,
0xb3, 0xc5, 0x99, 0x0a, 0x17, 0x34, 0x79, 0x53, 0xe1, 0xa2, 0xce, 0xe8, 0xdc, 0x22, 0xbf, 0x87,
0xe5, 0xf9, 0x56, 0x43, 0x3e, 0x9c, 0x77, 0x5b, 0xad, 0x83, 0x8d, 0x9d, 0xeb, 0x44, 0x4a, 0xe5,
0xaf, 0x00, 0x66, 0x1d, 0x84, 0x18, 0x35, 0x5b, 0xeb, 0x60, 0xe3, 0x8d, 0xc5, 0xcc, 0x52, 0xd5,
0x1f, 0x61, 0x6d, 0x21, 0x4c, 0x13, 0xa3, 0x4c, 0xae, 0x03, 0xfa, 0xf1, 0x8f, 0x6f, 0x94, 0x2b,
0xef, 0xfa, 0x0a, 0xee, 0xcc, 0xc1, 0x24, 0xd9, 0xaa, 0x56, 0x4d, 0x1d, 0x41, 0xc7, 0x0f, 0xcc,
0x7f, 0x4f, 0x0b, 0xc0, 0x4e, 0x55, 0xd6, 0x8b, 0xfb, 0xb0, 0x2c, 0x34, 0x44, 0x1c, 0x8b, 0x6d,
0x8d, 0xae, 0x2f, 0x00, 0x6d, 0x79, 0xcb, 0x13, 0x99, 0x1c, 0x75, 0xf0, 0x57, 0xe7, 0xcf, 0xfe,
0x1f, 0x00, 0x00, 0xff, 0xff, 0xbe, 0x11, 0xf3, 0xf2, 0xf9, 0x14, 0x00, 0x00,
}

20
weed/pb/master_pb/master.pb.go

@ -428,12 +428,10 @@ type VolumeEcShardInformationMessage struct {
EcIndexBits uint32 `protobuf:"varint,3,opt,name=ec_index_bits,json=ecIndexBits" json:"ec_index_bits,omitempty"`
}
func (m *VolumeEcShardInformationMessage) Reset() { *m = VolumeEcShardInformationMessage{} }
func (m *VolumeEcShardInformationMessage) String() string { return proto.CompactTextString(m) }
func (*VolumeEcShardInformationMessage) ProtoMessage() {}
func (*VolumeEcShardInformationMessage) Descriptor() ([]byte, []int) {
return fileDescriptor0, []int{4}
}
func (m *VolumeEcShardInformationMessage) Reset() { *m = VolumeEcShardInformationMessage{} }
func (m *VolumeEcShardInformationMessage) String() string { return proto.CompactTextString(m) }
func (*VolumeEcShardInformationMessage) ProtoMessage() {}
func (*VolumeEcShardInformationMessage) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} }
func (m *VolumeEcShardInformationMessage) GetId() uint32 {
if m != nil {
@ -1424,12 +1422,10 @@ type GetMasterConfigurationResponse struct {
MetricsIntervalSeconds uint32 `protobuf:"varint,2,opt,name=metrics_interval_seconds,json=metricsIntervalSeconds" json:"metrics_interval_seconds,omitempty"`
}
func (m *GetMasterConfigurationResponse) Reset() { *m = GetMasterConfigurationResponse{} }
func (m *GetMasterConfigurationResponse) String() string { return proto.CompactTextString(m) }
func (*GetMasterConfigurationResponse) ProtoMessage() {}
func (*GetMasterConfigurationResponse) Descriptor() ([]byte, []int) {
return fileDescriptor0, []int{32}
}
func (m *GetMasterConfigurationResponse) Reset() { *m = GetMasterConfigurationResponse{} }
func (m *GetMasterConfigurationResponse) String() string { return proto.CompactTextString(m) }
func (*GetMasterConfigurationResponse) ProtoMessage() {}
func (*GetMasterConfigurationResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{32} }
func (m *GetMasterConfigurationResponse) GetMetricsAddress() string {
if m != nil {

1
weed/pb/volume_server.proto

@ -94,6 +94,7 @@ service VolumeServer {
message BatchDeleteRequest {
repeated string file_ids = 1;
bool skip_cookie_check = 2;
}
message BatchDeleteResponse {

26
weed/pb/volume_server_pb/volume_server.pb.go

@ -1051,12 +1051,10 @@ func (m *VolumeEcShardsGenerateRequest) GetCollection() string {
type VolumeEcShardsGenerateResponse struct {
}
func (m *VolumeEcShardsGenerateResponse) Reset() { *m = VolumeEcShardsGenerateResponse{} }
func (m *VolumeEcShardsGenerateResponse) String() string { return proto.CompactTextString(m) }
func (*VolumeEcShardsGenerateResponse) ProtoMessage() {}
func (*VolumeEcShardsGenerateResponse) Descriptor() ([]byte, []int) {
return fileDescriptor0, []int{41}
}
func (m *VolumeEcShardsGenerateResponse) Reset() { *m = VolumeEcShardsGenerateResponse{} }
func (m *VolumeEcShardsGenerateResponse) String() string { return proto.CompactTextString(m) }
func (*VolumeEcShardsGenerateResponse) ProtoMessage() {}
func (*VolumeEcShardsGenerateResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{41} }
type VolumeEcShardsRebuildRequest struct {
VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId" json:"volume_id,omitempty"`
@ -1429,12 +1427,10 @@ func (m *VolumeEcShardsToVolumeRequest) GetCollection() string {
type VolumeEcShardsToVolumeResponse struct {
}
func (m *VolumeEcShardsToVolumeResponse) Reset() { *m = VolumeEcShardsToVolumeResponse{} }
func (m *VolumeEcShardsToVolumeResponse) String() string { return proto.CompactTextString(m) }
func (*VolumeEcShardsToVolumeResponse) ProtoMessage() {}
func (*VolumeEcShardsToVolumeResponse) Descriptor() ([]byte, []int) {
return fileDescriptor0, []int{57}
}
func (m *VolumeEcShardsToVolumeResponse) Reset() { *m = VolumeEcShardsToVolumeResponse{} }
func (m *VolumeEcShardsToVolumeResponse) String() string { return proto.CompactTextString(m) }
func (*VolumeEcShardsToVolumeResponse) ProtoMessage() {}
func (*VolumeEcShardsToVolumeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{57} }
type ReadVolumeFileStatusRequest struct {
VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId" json:"volume_id,omitempty"`
@ -2101,10 +2097,8 @@ type QueryRequest_InputSerialization_JSONInput struct {
func (m *QueryRequest_InputSerialization_JSONInput) Reset() {
*m = QueryRequest_InputSerialization_JSONInput{}
}
func (m *QueryRequest_InputSerialization_JSONInput) String() string {
return proto.CompactTextString(m)
}
func (*QueryRequest_InputSerialization_JSONInput) ProtoMessage() {}
func (m *QueryRequest_InputSerialization_JSONInput) String() string { return proto.CompactTextString(m) }
func (*QueryRequest_InputSerialization_JSONInput) ProtoMessage() {}
func (*QueryRequest_InputSerialization_JSONInput) Descriptor() ([]byte, []int) {
return fileDescriptor0, []int{70, 1, 1}
}

70
weed/server/filer_grpc_server_listen.go

@ -0,0 +1,70 @@
package weed_server
import (
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
)
func (fs *FilerServer) ListenForEvents(req *filer_pb.ListenForEventsRequest, stream filer_pb.SeaweedFiler_ListenForEventsServer) error {
peerAddress := findClientAddress(stream.Context(), 0)
clientName, messageChan := fs.addClient(req.ClientName, peerAddress)
defer fs.deleteClient(clientName, messageChan)
// ts := time.Unix(req.SinceSec, 0)
// iterate through old messages
/*
for _, message := range ms.Topo.ToVolumeLocations() {
if err := stream.Send(message); err != nil {
return err
}
}
*/
// need to add a buffer here to avoid slow clients
// also needs to support millions of clients
for message := range messageChan {
if err := stream.Send(message); err != nil {
glog.V(0).Infof("=> client %v: %+v", clientName, message)
return err
}
}
return nil
}
func (fs *FilerServer) addClient(clientType string, clientAddress string) (clientName string, messageChan chan *filer_pb.FullEventNotification) {
clientName = clientType + "@" + clientAddress
glog.V(0).Infof("+ listener %v", clientName)
messageChan = make(chan *filer_pb.FullEventNotification, 10)
fs.clientChansLock.Lock()
fs.clientChans[clientName] = messageChan
fs.clientChansLock.Unlock()
return
}
func (fs *FilerServer) deleteClient(clientName string, messageChan chan *filer_pb.FullEventNotification) {
glog.V(0).Infof("- listener %v", clientName)
close(messageChan)
fs.clientChansLock.Lock()
delete(fs.clientChans, clientName)
fs.clientChansLock.Unlock()
}
func (fs *FilerServer) sendMessageToClients(dir string, eventNotification *filer_pb.EventNotification) {
message := &filer_pb.FullEventNotification{
Directory: dir,
EventNotification: eventNotification,
}
fs.clientChansLock.RLock()
for _, ch := range fs.clientChans {
ch <- message
}
fs.clientChansLock.RUnlock()
}

7
weed/server/filer_server.go

@ -5,11 +5,13 @@ import (
"fmt"
"net/http"
"os"
"sync"
"time"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/stats"
"github.com/chrislusf/seaweedfs/weed/util"
@ -52,6 +54,11 @@ type FilerServer struct {
secret security.SigningKey
filer *filer2.Filer
grpcDialOption grpc.DialOption
// notifying clients
clientChansLock sync.RWMutex
clientChans map[string]chan *filer_pb.FullEventNotification
}
func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption) (fs *FilerServer, err error) {

17
weed/util/bytes.go

@ -74,3 +74,20 @@ func HashStringToLong(dir string) (v int64) {
return
}
func HashToInt32(data []byte) (v int32) {
h := md5.New()
h.Write(data)
b := h.Sum(nil)
v += int32(b[0])
v <<= 8
v += int32(b[1])
v <<= 8
v += int32(b[2])
v <<= 8
v += int32(b[3])
return
}
Loading…
Cancel
Save