From dfccc3c2637693dce141c27a321ba5d3aea1ace9 Mon Sep 17 00:00:00 2001 From: Chris Lu Date: Fri, 8 May 2020 02:47:22 -0700 Subject: [PATCH] able to read chan and write chan --- go.mod | 2 + go.sum | 2 + other/java/client/src/main/proto/filer.proto | 19 + weed/messaging/broker/broker_append.go | 4 + weed/messaging/broker/broker_grpc_server.go | 4 + .../broker/broker_grpc_server_discovery.go | 37 +- .../broker/broker_grpc_server_publish.go | 30 +- .../broker/broker_grpc_server_subscribe.go | 12 +- weed/messaging/broker/broker_server.go | 24 +- .../broker/consistent_distribution.go | 38 ++ .../broker/consistent_distribution_test.go | 32 ++ weed/messaging/broker/topic_lock.go | 17 + weed/messaging/msgclient/pub_sub_chan.go | 38 +- weed/messaging/msgclient/publisher.go | 11 +- weed/messaging/msgclient/subscriber.go | 12 +- weed/pb/filer.proto | 19 + weed/pb/filer_pb/filer.pb.go | 381 ++++++++++++------ weed/pb/messaging.proto | 14 +- weed/pb/messaging_pb/messaging.pb.go | 248 ++++++++---- weed/server/filer_grpc_server.go | 38 +- weed/server/filer_grpc_server_listen.go | 2 +- weed/server/filer_server.go | 4 +- weed/util/log_buffer/log_read.go | 6 +- 23 files changed, 734 insertions(+), 260 deletions(-) create mode 100644 weed/messaging/broker/consistent_distribution.go create mode 100644 weed/messaging/broker/consistent_distribution_test.go diff --git a/go.mod b/go.mod index 96ea8fd3d..2ba8658d3 100644 --- a/go.mod +++ b/go.mod @@ -10,6 +10,8 @@ require ( github.com/OneOfOne/xxhash v1.2.2 github.com/Shopify/sarama v1.23.1 github.com/aws/aws-sdk-go v1.23.13 + github.com/buraksezer/consistent v0.0.0-20191006190839-693edf70fd72 + github.com/cespare/xxhash v1.1.0 github.com/chrislusf/raft v0.0.0-20190225081310-10d6e2182d92 github.com/coreos/bbolt v1.3.3 // indirect github.com/coreos/etcd v3.3.15+incompatible // indirect diff --git a/go.sum b/go.sum index a382daf10..00329907e 100644 --- a/go.sum +++ b/go.sum @@ -58,6 +58,8 @@ github.com/bitly/go-hostpool v0.0.0-20171023180738-a3a6125de932 h1:mXoPYz/Ul5HYE github.com/bitly/go-hostpool v0.0.0-20171023180738-a3a6125de932/go.mod h1:NOuUCSz6Q9T7+igc/hlvDOUdtWKryOrtFyIVABv/p7k= github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4YnC6+E63dPcxHo2sUxDIu8g3QgEJdRY= github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4= +github.com/buraksezer/consistent v0.0.0-20191006190839-693edf70fd72 h1:fUmDBbSvv1uOzo/t8WaxZMVb7BxJ8JECo5lGoR9c5bA= +github.com/buraksezer/consistent v0.0.0-20191006190839-693edf70fd72/go.mod h1:OEE5igu/CDjGegM1Jn6ZMo7R6LlV/JChAkjfQQIRLpg= github.com/census-instrumentation/opencensus-proto v0.2.0 h1:LzQXZOgg4CQfE6bFvXGM30YZL1WW/M337pXml+GrcZ4= github.com/census-instrumentation/opencensus-proto v0.2.0/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/census-instrumentation/opencensus-proto v0.2.1 h1:glEXhBS5PSLLv4IXzLA5yPRVX4bilULVyxxbrfOtDAk= diff --git a/other/java/client/src/main/proto/filer.proto b/other/java/client/src/main/proto/filer.proto index 3b3b78bbb..1fc8ef63d 100644 --- a/other/java/client/src/main/proto/filer.proto +++ b/other/java/client/src/main/proto/filer.proto @@ -50,6 +50,10 @@ service SeaweedFiler { rpc KeepConnected (stream KeepConnectedRequest) returns (stream KeepConnectedResponse) { } + + rpc LocateBroker (LocateBrokerRequest) returns (LocateBrokerResponse) { + } + } ////////////////////////////////////////////////// @@ -267,6 +271,21 @@ message LogEntry { message KeepConnectedRequest { string name = 1; uint32 grpc_port = 2; + repeated string resources = 3; } message KeepConnectedResponse { } + +message LocateBrokerRequest { + string resource = 1; +} +message LocateBrokerResponse { + bool found = 1; + // if found, send the exact address + // if not found, send the full list of existing brokers + message Resource { + string grpc_addresses = 1; + int32 resource_count = 2; + } + repeated Resource resources = 2; +} diff --git a/weed/messaging/broker/broker_append.go b/weed/messaging/broker/broker_append.go index e87e197b0..80f107e00 100644 --- a/weed/messaging/broker/broker_append.go +++ b/weed/messaging/broker/broker_append.go @@ -3,6 +3,7 @@ package broker import ( "context" "fmt" + "io" "github.com/chrislusf/seaweedfs/weed/glog" "github.com/chrislusf/seaweedfs/weed/operation" @@ -94,6 +95,9 @@ func (broker *MessageBroker) WithFilerClient(fn func(filer_pb.SeaweedFilerClient for _, filer := range broker.option.Filers { if err = pb.WithFilerClient(filer, broker.grpcDialOption, fn); err != nil { + if err == io.EOF { + return + } glog.V(0).Infof("fail to connect to %s: %v", filer, err) } else { break diff --git a/weed/messaging/broker/broker_grpc_server.go b/weed/messaging/broker/broker_grpc_server.go index 447620a6b..32dab6813 100644 --- a/weed/messaging/broker/broker_grpc_server.go +++ b/weed/messaging/broker/broker_grpc_server.go @@ -10,6 +10,10 @@ func (broker *MessageBroker) ConfigureTopic(c context.Context, request *messagin panic("implement me") } +func (broker *MessageBroker) DeleteTopic(c context.Context, request *messaging_pb.DeleteTopicRequest) (*messaging_pb.DeleteTopicResponse, error) { + panic("implement me") +} + func (broker *MessageBroker) GetTopicConfiguration(c context.Context, request *messaging_pb.GetTopicConfigurationRequest) (*messaging_pb.GetTopicConfigurationResponse, error) { panic("implement me") } diff --git a/weed/messaging/broker/broker_grpc_server_discovery.go b/weed/messaging/broker/broker_grpc_server_discovery.go index 4b7f357fa..3c14f3220 100644 --- a/weed/messaging/broker/broker_grpc_server_discovery.go +++ b/weed/messaging/broker/broker_grpc_server_discovery.go @@ -2,6 +2,7 @@ package broker import ( "context" + "fmt" "time" "github.com/chrislusf/seaweedfs/weed/glog" @@ -25,12 +26,40 @@ If one of the pub or sub connects very late, and the system topo changed quite a func (broker *MessageBroker) FindBroker(c context.Context, request *messaging_pb.FindBrokerRequest) (*messaging_pb.FindBrokerResponse, error) { - panic("implement me") + t := &messaging_pb.FindBrokerResponse{} + var peers []string + + targetTopicPartition := fmt.Sprintf(TopicPartitionFmt, request.Namespace, request.Topic, request.Parition) + + for _, filer := range broker.option.Filers { + err := broker.withFilerClient(filer, func(client filer_pb.SeaweedFilerClient) error { + resp, err := client.LocateBroker(context.Background(), &filer_pb.LocateBrokerRequest{ + Resource: targetTopicPartition, + }) + if err != nil { + return err + } + if resp.Found && len(resp.Resources) > 0 { + t.Broker = resp.Resources[0].GrpcAddresses + return nil + } + for _, b := range resp.Resources { + peers = append(peers, b.GrpcAddresses) + } + return nil + }) + if err != nil { + return nil, err + } + } + + t.Broker = PickMember(peers, []byte(targetTopicPartition)) + + return t, nil + } - - -func (broker *MessageBroker) checkPeers() { +func (broker *MessageBroker) checkFilers() { // contact a filer about masters var masters []string diff --git a/weed/messaging/broker/broker_grpc_server_publish.go b/weed/messaging/broker/broker_grpc_server_publish.go index b3a909a6c..61e53b433 100644 --- a/weed/messaging/broker/broker_grpc_server_publish.go +++ b/weed/messaging/broker/broker_grpc_server_publish.go @@ -47,24 +47,11 @@ func (broker *MessageBroker) Publish(stream messaging_pb.SeaweedMessaging_Publis tl := broker.topicLocks.RequestLock(tp, topicConfig, true) defer broker.topicLocks.ReleaseLock(tp, true) - updatesChan := make(chan int32) - - go func() { - for update := range updatesChan { - if err := stream.Send(&messaging_pb.PublishResponse{ - Config: &messaging_pb.PublishResponse_ConfigMessage{ - PartitionCount: update, - }, - }); err != nil { - glog.V(0).Infof("err sending publish response: %v", err) - return - } - } - }() - // process each message for { + // println("recv") in, err := stream.Recv() + // glog.V(0).Infof("recieved %v err: %v", in, err) if err == io.EOF { return nil } @@ -86,5 +73,18 @@ func (broker *MessageBroker) Publish(stream messaging_pb.SeaweedMessaging_Publis tl.logBuffer.AddToBuffer(in.Data.Key, data) + if in.Data.IsClose { + // println("server received closing") + break + } + } + + // send the close ack + // println("server send ack closing") + if err := stream.Send(&messaging_pb.PublishResponse{IsClosed: true}); err != nil { + glog.V(0).Infof("err sending close response: %v", err) + } + return nil + } diff --git a/weed/messaging/broker/broker_grpc_server_subscribe.go b/weed/messaging/broker/broker_grpc_server_subscribe.go index 472a5007b..761129e80 100644 --- a/weed/messaging/broker/broker_grpc_server_subscribe.go +++ b/weed/messaging/broker/broker_grpc_server_subscribe.go @@ -83,11 +83,17 @@ func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_Subs glog.Errorf("sending %d bytes to %s: %s", len(m.Value), subscriberId, err) return err } + if m.IsClose { + // println("processed EOF") + return io.EOF + } processedTsNs = logEntry.TsNs + messageCount++ return nil } if err := broker.readPersistedLogBuffer(&tp, lastReadTime, eachLogEntryFn); err != nil { + // println("stopping from persisted logs") return err } @@ -95,7 +101,7 @@ func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_Subs lastReadTime = time.Unix(0, processedTsNs) } - messageCount, err = lock.logBuffer.LoopProcessLogData(lastReadTime, func() bool { + err = lock.logBuffer.LoopProcessLogData(lastReadTime, func() bool { lock.Mutex.Lock() lock.cond.Wait() lock.Mutex.Unlock() @@ -124,7 +130,7 @@ func (broker *MessageBroker) readPersistedLogBuffer(tp *TopicPartition, startTim return nil } } - if !strings.HasSuffix(hourMinuteEntry.Name, partitionSuffix){ + if !strings.HasSuffix(hourMinuteEntry.Name, partitionSuffix) { return nil } // println("partition", tp.Partition, "processing", dayDir, "/", hourMinuteEntry.Name) @@ -133,7 +139,7 @@ func (broker *MessageBroker) readPersistedLogBuffer(tp *TopicPartition, startTim if err := filer2.ReadEachLogEntry(chunkedFileReader, sizeBuf, startTsNs, eachLogEntryFn); err != nil { chunkedFileReader.Close() if err == io.EOF { - return nil + return err } return fmt.Errorf("reading %s/%s: %v", dayDir, hourMinuteEntry.Name, err) } diff --git a/weed/messaging/broker/broker_server.go b/weed/messaging/broker/broker_server.go index 9cad27214..e6ff2cf00 100644 --- a/weed/messaging/broker/broker_server.go +++ b/weed/messaging/broker/broker_server.go @@ -36,7 +36,7 @@ func NewMessageBroker(option *MessageBrokerOption, grpcDialOption grpc.DialOptio messageBroker.topicLocks = NewTopicLocks(messageBroker) - messageBroker.checkPeers() + messageBroker.checkFilers() go messageBroker.keepConnectedToOneFiler() @@ -53,6 +53,24 @@ func (broker *MessageBroker) keepConnectedToOneFiler() { glog.V(0).Infof("%s:%d failed to keep connected to %s: %v", broker.option.Ip, broker.option.Port, filer, err) return err } + + initRequest := &filer_pb.KeepConnectedRequest{ + Name: broker.option.Ip, + GrpcPort: uint32(broker.option.Port), + } + for _, tp := range broker.topicLocks.ListTopicPartitions() { + initRequest.Resources = append(initRequest.Resources, tp.String()) + } + if err := stream.Send(&filer_pb.KeepConnectedRequest{ + Name: broker.option.Ip, + GrpcPort: uint32(broker.option.Port), + }); err != nil { + glog.V(0).Infof("broker %s:%d failed to init at %s: %v", broker.option.Ip, broker.option.Port, filer, err) + return err + } + + // TODO send events of adding/removing topics + glog.V(0).Infof("conntected with filer: %v", filer) for { if err := stream.Send(&filer_pb.KeepConnectedRequest{ @@ -68,12 +86,12 @@ func (broker *MessageBroker) keepConnectedToOneFiler() { return err } // println("received reply") - time.Sleep(11*time.Second) + time.Sleep(11 * time.Second) // println("woke up") } return nil }) - time.Sleep(3*time.Second) + time.Sleep(3 * time.Second) } } diff --git a/weed/messaging/broker/consistent_distribution.go b/weed/messaging/broker/consistent_distribution.go new file mode 100644 index 000000000..dd7d34f86 --- /dev/null +++ b/weed/messaging/broker/consistent_distribution.go @@ -0,0 +1,38 @@ +package broker + +import ( + "github.com/cespare/xxhash" + "github.com/buraksezer/consistent" +) + +type Member string + +func (m Member) String() string { + return string(m) +} + +type hasher struct{} + +func (h hasher) Sum64(data []byte) uint64 { + return xxhash.Sum64(data) +} + +func PickMember(members []string, key []byte) string { + cfg := consistent.Config{ + PartitionCount: 9791, + ReplicationFactor: 2, + Load: 1.25, + Hasher: hasher{}, + } + + cmembers := []consistent.Member{} + for _, m := range members { + cmembers = append(cmembers, Member(m)) + } + + c := consistent.New(cmembers, cfg) + + m := c.LocateKey(key) + + return m.String() +} \ No newline at end of file diff --git a/weed/messaging/broker/consistent_distribution_test.go b/weed/messaging/broker/consistent_distribution_test.go new file mode 100644 index 000000000..192516092 --- /dev/null +++ b/weed/messaging/broker/consistent_distribution_test.go @@ -0,0 +1,32 @@ +package broker + +import ( + "fmt" + "testing" +) + +func TestPickMember(t *testing.T) { + + servers := []string{ + "s1:port", + "s2:port", + "s3:port", + "s5:port", + "s4:port", + } + + total := 1000 + + distribution := make(map[string]int) + for i:=0;i", m) + distribution[m]++ + } + + for member, count := range distribution { + fmt.Printf("member: %s, key count: %d load=%.2f\n", member, count, float64(count*100)/float64(total/len(servers))) + } + +} \ No newline at end of file diff --git a/weed/messaging/broker/topic_lock.go b/weed/messaging/broker/topic_lock.go index f8a5aa171..f3a66a2f5 100644 --- a/weed/messaging/broker/topic_lock.go +++ b/weed/messaging/broker/topic_lock.go @@ -16,6 +16,13 @@ type TopicPartition struct { Topic string Partition int32 } +const ( + TopicPartitionFmt = "%s/%s_%2d" +) +func (tp *TopicPartition) String() string { + return fmt.Sprintf(TopicPartitionFmt, tp.Namespace, tp.Topic, tp.Partition) +} + type TopicLock struct { sync.Mutex cond *sync.Cond @@ -101,3 +108,13 @@ func (tl *TopicLocks) ReleaseLock(partition TopicPartition, isPublisher bool) { delete(tl.locks, partition) } } + +func (tl *TopicLocks) ListTopicPartitions() (tps []TopicPartition) { + tl.Lock() + defer tl.Unlock() + + for k := range tl.locks { + tps = append(tps, k) + } + return +} diff --git a/weed/messaging/msgclient/pub_sub_chan.go b/weed/messaging/msgclient/pub_sub_chan.go index d39e4c658..a11240080 100644 --- a/weed/messaging/msgclient/pub_sub_chan.go +++ b/weed/messaging/msgclient/pub_sub_chan.go @@ -5,12 +5,15 @@ import ( "log" "time" + "google.golang.org/grpc" + "github.com/chrislusf/seaweedfs/weed/messaging/broker" "github.com/chrislusf/seaweedfs/weed/pb/messaging_pb" ) type PubChannel struct { - client messaging_pb.SeaweedMessaging_PublishClient + client messaging_pb.SeaweedMessaging_PublishClient + grpcConnection *grpc.ClientConn } func (mc *MessagingClient) NewPubChannel(chanName string) (*PubChannel, error) { @@ -28,7 +31,8 @@ func (mc *MessagingClient) NewPubChannel(chanName string) (*PubChannel, error) { return nil, err } return &PubChannel{ - client: pc, + client: pc, + grpcConnection: grpcConnection, }, nil } @@ -40,7 +44,24 @@ func (pc *PubChannel) Publish(m []byte) error { }) } func (pc *PubChannel) Close() error { - return pc.client.CloseSend() + + // println("send closing") + if err := pc.client.Send(&messaging_pb.PublishRequest{ + Data: &messaging_pb.Message{ + IsClose: true, + }, + }); err != nil { + log.Printf("err send close: %v", err) + } + // println("receive closing") + if _, err := pc.client.Recv(); err != nil && err != io.EOF { + log.Printf("err receive close: %v", err) + } + // println("close connection") + if err := pc.grpcConnection.Close(); err != nil { + log.Printf("err connection close: %v", err) + } + return nil } type SubChannel struct { @@ -58,7 +79,7 @@ func (mc *MessagingClient) NewSubChannel(chanName string) (*SubChannel, error) { if err != nil { return nil, err } - sc, err := setupSubscriberClient(grpcConnection, "", "chan", chanName, 0, time.Unix(0,0)) + sc, err := setupSubscriberClient(grpcConnection, "", "chan", chanName, 0, time.Unix(0, 0)) if err != nil { return nil, err } @@ -78,13 +99,14 @@ func (mc *MessagingClient) NewSubChannel(chanName string) (*SubChannel, error) { log.Printf("fail to receive from netchan %s: %v", chanName, subErr) return } - if resp.IsClose { + if resp.Data.IsClose { + t.stream.Send(&messaging_pb.SubscriberMessage{ + IsClose: true, + }) close(t.ch) return } - if resp.Data != nil { - t.ch <- resp.Data.Value - } + t.ch <- resp.Data.Value } }() diff --git a/weed/messaging/msgclient/publisher.go b/weed/messaging/msgclient/publisher.go index b0459494b..08f1d278a 100644 --- a/weed/messaging/msgclient/publisher.go +++ b/weed/messaging/msgclient/publisher.go @@ -4,9 +4,9 @@ import ( "context" "github.com/OneOfOne/xxhash" + "google.golang.org/grpc" "github.com/chrislusf/seaweedfs/weed/messaging/broker" - "github.com/chrislusf/seaweedfs/weed/pb" "github.com/chrislusf/seaweedfs/weed/pb/messaging_pb" ) @@ -16,7 +16,7 @@ type Publisher struct { messageCount uint64 publisherId string } - +/* func (mc *MessagingClient) NewPublisher(publisherId, namespace, topic string) (*Publisher, error) { // read topic configuration topicConfiguration := &messaging_pb.TopicConfiguration{ @@ -24,7 +24,11 @@ func (mc *MessagingClient) NewPublisher(publisherId, namespace, topic string) (* } publishClients := make([]messaging_pb.SeaweedMessaging_PublishClient, topicConfiguration.PartitionCount) for i := 0; i < int(topicConfiguration.PartitionCount); i++ { - client, err := mc.setupPublisherClient(namespace, topic, int32(i)) + client, err := setupPublisherClient(broker.TopicPartition{ + Namespace: namespace, + Topic: topic, + Partition: int32(i), + }) if err != nil { return nil, err } @@ -35,6 +39,7 @@ func (mc *MessagingClient) NewPublisher(publisherId, namespace, topic string) (* topicConfiguration: topicConfiguration, }, nil } +*/ func setupPublisherClient(grpcConnection *grpc.ClientConn, tp broker.TopicPartition) (messaging_pb.SeaweedMessaging_PublishClient, error) { diff --git a/weed/messaging/msgclient/subscriber.go b/weed/messaging/msgclient/subscriber.go index 27fa35a5b..d3066d6ef 100644 --- a/weed/messaging/msgclient/subscriber.go +++ b/weed/messaging/msgclient/subscriber.go @@ -5,6 +5,7 @@ import ( "io" "time" + "google.golang.org/grpc" "github.com/chrislusf/seaweedfs/weed/pb/messaging_pb" ) @@ -13,6 +14,7 @@ type Subscriber struct { subscriberId string } +/* func (mc *MessagingClient) NewSubscriber(subscriberId, namespace, topic string, startTime time.Time) (*Subscriber, error) { // read topic configuration topicConfiguration := &messaging_pb.TopicConfiguration{ @@ -36,9 +38,9 @@ func (mc *MessagingClient) NewSubscriber(subscriberId, namespace, topic string, func (mc *MessagingClient) setupSubscriberClient(subscriberId, namespace, topic string, partition int32, startTime time.Time) (messaging_pb.SeaweedMessaging_SubscribeClient, error) { - stream, newBroker, err := mc.initSubscriberClient(subscriberId, namespace, topic, partition, startTime) + stream, err := setupSubscriberClient(subscriberId, namespace, topic, partition, startTime) if err != nil { - return client, err + return stream, err } if newBroker != nil { @@ -47,6 +49,7 @@ func (mc *MessagingClient) setupSubscriberClient(subscriberId, namespace, topic return stream, nil } +*/ func setupSubscriberClient(grpcConnection *grpc.ClientConn, subscriberId string, namespace string, topic string, partition int32, startTime time.Time) (stream messaging_pb.SeaweedMessaging_SubscribeClient, err error) { stream, err = messaging_pb.NewSeaweedMessagingClient(grpcConnection).Subscribe(context.Background()) @@ -70,13 +73,10 @@ func setupSubscriberClient(grpcConnection *grpc.ClientConn, subscriberId string, } // process init response - initResponse, err := stream.Recv() + _, err = stream.Recv() if err != nil { return } - if initResponse.Redirect != nil { - // TODO follow redirection - } return stream, nil } diff --git a/weed/pb/filer.proto b/weed/pb/filer.proto index 3b3b78bbb..1fc8ef63d 100644 --- a/weed/pb/filer.proto +++ b/weed/pb/filer.proto @@ -50,6 +50,10 @@ service SeaweedFiler { rpc KeepConnected (stream KeepConnectedRequest) returns (stream KeepConnectedResponse) { } + + rpc LocateBroker (LocateBrokerRequest) returns (LocateBrokerResponse) { + } + } ////////////////////////////////////////////////// @@ -267,6 +271,21 @@ message LogEntry { message KeepConnectedRequest { string name = 1; uint32 grpc_port = 2; + repeated string resources = 3; } message KeepConnectedResponse { } + +message LocateBrokerRequest { + string resource = 1; +} +message LocateBrokerResponse { + bool found = 1; + // if found, send the exact address + // if not found, send the full list of existing brokers + message Resource { + string grpc_addresses = 1; + int32 resource_count = 2; + } + repeated Resource resources = 2; +} diff --git a/weed/pb/filer_pb/filer.pb.go b/weed/pb/filer_pb/filer.pb.go index f1320273f..f5b62e377 100644 --- a/weed/pb/filer_pb/filer.pb.go +++ b/weed/pb/filer_pb/filer.pb.go @@ -46,6 +46,8 @@ It has these top-level messages: LogEntry KeepConnectedRequest KeepConnectedResponse + LocateBrokerRequest + LocateBrokerResponse */ package filer_pb @@ -1231,8 +1233,9 @@ func (m *LogEntry) GetData() []byte { } type KeepConnectedRequest struct { - Name string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"` - GrpcPort uint32 `protobuf:"varint,2,opt,name=grpc_port,json=grpcPort" json:"grpc_port,omitempty"` + Name string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"` + GrpcPort uint32 `protobuf:"varint,2,opt,name=grpc_port,json=grpcPort" json:"grpc_port,omitempty"` + Resources []string `protobuf:"bytes,3,rep,name=resources" json:"resources,omitempty"` } func (m *KeepConnectedRequest) Reset() { *m = KeepConnectedRequest{} } @@ -1254,6 +1257,13 @@ func (m *KeepConnectedRequest) GetGrpcPort() uint32 { return 0 } +func (m *KeepConnectedRequest) GetResources() []string { + if m != nil { + return m.Resources + } + return nil +} + type KeepConnectedResponse struct { } @@ -1262,6 +1272,74 @@ func (m *KeepConnectedResponse) String() string { return proto.Compac func (*KeepConnectedResponse) ProtoMessage() {} func (*KeepConnectedResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{36} } +type LocateBrokerRequest struct { + Resource string `protobuf:"bytes,1,opt,name=resource" json:"resource,omitempty"` +} + +func (m *LocateBrokerRequest) Reset() { *m = LocateBrokerRequest{} } +func (m *LocateBrokerRequest) String() string { return proto.CompactTextString(m) } +func (*LocateBrokerRequest) ProtoMessage() {} +func (*LocateBrokerRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{37} } + +func (m *LocateBrokerRequest) GetResource() string { + if m != nil { + return m.Resource + } + return "" +} + +type LocateBrokerResponse struct { + Found bool `protobuf:"varint,1,opt,name=found" json:"found,omitempty"` + Resources []*LocateBrokerResponse_Resource `protobuf:"bytes,2,rep,name=resources" json:"resources,omitempty"` +} + +func (m *LocateBrokerResponse) Reset() { *m = LocateBrokerResponse{} } +func (m *LocateBrokerResponse) String() string { return proto.CompactTextString(m) } +func (*LocateBrokerResponse) ProtoMessage() {} +func (*LocateBrokerResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{38} } + +func (m *LocateBrokerResponse) GetFound() bool { + if m != nil { + return m.Found + } + return false +} + +func (m *LocateBrokerResponse) GetResources() []*LocateBrokerResponse_Resource { + if m != nil { + return m.Resources + } + return nil +} + +// if found, send the exact address +// if not found, send the full list of existing brokers +type LocateBrokerResponse_Resource struct { + GrpcAddresses string `protobuf:"bytes,1,opt,name=grpc_addresses,json=grpcAddresses" json:"grpc_addresses,omitempty"` + ResourceCount int32 `protobuf:"varint,2,opt,name=resource_count,json=resourceCount" json:"resource_count,omitempty"` +} + +func (m *LocateBrokerResponse_Resource) Reset() { *m = LocateBrokerResponse_Resource{} } +func (m *LocateBrokerResponse_Resource) String() string { return proto.CompactTextString(m) } +func (*LocateBrokerResponse_Resource) ProtoMessage() {} +func (*LocateBrokerResponse_Resource) Descriptor() ([]byte, []int) { + return fileDescriptor0, []int{38, 0} +} + +func (m *LocateBrokerResponse_Resource) GetGrpcAddresses() string { + if m != nil { + return m.GrpcAddresses + } + return "" +} + +func (m *LocateBrokerResponse_Resource) GetResourceCount() int32 { + if m != nil { + return m.ResourceCount + } + return 0 +} + func init() { proto.RegisterType((*LookupDirectoryEntryRequest)(nil), "filer_pb.LookupDirectoryEntryRequest") proto.RegisterType((*LookupDirectoryEntryResponse)(nil), "filer_pb.LookupDirectoryEntryResponse") @@ -1300,6 +1378,9 @@ func init() { proto.RegisterType((*LogEntry)(nil), "filer_pb.LogEntry") proto.RegisterType((*KeepConnectedRequest)(nil), "filer_pb.KeepConnectedRequest") proto.RegisterType((*KeepConnectedResponse)(nil), "filer_pb.KeepConnectedResponse") + proto.RegisterType((*LocateBrokerRequest)(nil), "filer_pb.LocateBrokerRequest") + proto.RegisterType((*LocateBrokerResponse)(nil), "filer_pb.LocateBrokerResponse") + proto.RegisterType((*LocateBrokerResponse_Resource)(nil), "filer_pb.LocateBrokerResponse.Resource") } // Reference imports to suppress errors if they are not otherwise used. @@ -1327,6 +1408,7 @@ type SeaweedFilerClient interface { GetFilerConfiguration(ctx context.Context, in *GetFilerConfigurationRequest, opts ...grpc.CallOption) (*GetFilerConfigurationResponse, error) SubscribeMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeMetadataClient, error) KeepConnected(ctx context.Context, opts ...grpc.CallOption) (SeaweedFiler_KeepConnectedClient, error) + LocateBroker(ctx context.Context, in *LocateBrokerRequest, opts ...grpc.CallOption) (*LocateBrokerResponse, error) } type seaweedFilerClient struct { @@ -1531,6 +1613,15 @@ func (x *seaweedFilerKeepConnectedClient) Recv() (*KeepConnectedResponse, error) return m, nil } +func (c *seaweedFilerClient) LocateBroker(ctx context.Context, in *LocateBrokerRequest, opts ...grpc.CallOption) (*LocateBrokerResponse, error) { + out := new(LocateBrokerResponse) + err := grpc.Invoke(ctx, "/filer_pb.SeaweedFiler/LocateBroker", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // Server API for SeaweedFiler service type SeaweedFilerServer interface { @@ -1548,6 +1639,7 @@ type SeaweedFilerServer interface { GetFilerConfiguration(context.Context, *GetFilerConfigurationRequest) (*GetFilerConfigurationResponse, error) SubscribeMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeMetadataServer) error KeepConnected(SeaweedFiler_KeepConnectedServer) error + LocateBroker(context.Context, *LocateBrokerRequest) (*LocateBrokerResponse, error) } func RegisterSeaweedFilerServer(s *grpc.Server, srv SeaweedFilerServer) { @@ -1820,6 +1912,24 @@ func (x *seaweedFilerKeepConnectedServer) Recv() (*KeepConnectedRequest, error) return m, nil } +func _SeaweedFiler_LocateBroker_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LocateBrokerRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).LocateBroker(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/LocateBroker", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).LocateBroker(ctx, req.(*LocateBrokerRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{ ServiceName: "filer_pb.SeaweedFiler", HandlerType: (*SeaweedFilerServer)(nil), @@ -1868,6 +1978,10 @@ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{ MethodName: "GetFilerConfiguration", Handler: _SeaweedFiler_GetFilerConfiguration_Handler, }, + { + MethodName: "LocateBroker", + Handler: _SeaweedFiler_LocateBroker_Handler, + }, }, Streams: []grpc.StreamDesc{ { @@ -1893,132 +2007,139 @@ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{ func init() { proto.RegisterFile("filer.proto", fileDescriptor0) } var fileDescriptor0 = []byte{ - // 2020 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x58, 0xcd, 0x6e, 0xdc, 0xc8, - 0x11, 0x16, 0x67, 0x34, 0xa3, 0x61, 0xcd, 0x8c, 0x57, 0x6a, 0x49, 0xf6, 0x68, 0xf4, 0x63, 0x2d, - 0x1d, 0x6f, 0x14, 0xd8, 0x50, 0x0c, 0x65, 0x03, 0xec, 0x66, 0x93, 0x83, 0x2d, 0xcb, 0x8e, 0x63, - 0x5b, 0x2b, 0x50, 0xf6, 0x22, 0x41, 0x80, 0x30, 0x14, 0xd9, 0x1a, 0x75, 0xc4, 0x21, 0x99, 0xee, - 0xa6, 0x7e, 0xf6, 0xb4, 0xcf, 0x11, 0x20, 0x6f, 0x91, 0x63, 0x90, 0x4b, 0x10, 0x20, 0x40, 0xce, - 0x79, 0x81, 0x3c, 0x49, 0xd0, 0xd5, 0x24, 0xa7, 0x39, 0x3f, 0xd2, 0x1a, 0x0b, 0xdf, 0xba, 0xab, - 0xaa, 0xab, 0xab, 0xeb, 0xe7, 0xab, 0x22, 0xa1, 0x7d, 0xca, 0x22, 0xca, 0x77, 0x53, 0x9e, 0xc8, - 0x84, 0xb4, 0x70, 0xe3, 0xa5, 0x27, 0xce, 0xd7, 0xb0, 0xfe, 0x26, 0x49, 0xce, 0xb3, 0xf4, 0x39, - 0xe3, 0x34, 0x90, 0x09, 0xbf, 0x3e, 0x88, 0x25, 0xbf, 0x76, 0xe9, 0x9f, 0x33, 0x2a, 0x24, 0xd9, - 0x00, 0x3b, 0x2c, 0x18, 0x3d, 0x6b, 0xdb, 0xda, 0xb1, 0xdd, 0x11, 0x81, 0x10, 0x98, 0x8f, 0xfd, - 0x21, 0xed, 0xd5, 0x90, 0x81, 0x6b, 0xe7, 0x00, 0x36, 0xa6, 0x2b, 0x14, 0x69, 0x12, 0x0b, 0x4a, - 0x1e, 0x42, 0x83, 0x2a, 0x02, 0x6a, 0x6b, 0xef, 0x7d, 0xb2, 0x5b, 0x98, 0xb2, 0xab, 0xe5, 0x34, - 0xd7, 0xf9, 0x87, 0x05, 0xe4, 0x0d, 0x13, 0x52, 0x11, 0x19, 0x15, 0xdf, 0xcf, 0x9e, 0xbb, 0xd0, - 0x4c, 0x39, 0x3d, 0x65, 0x57, 0xb9, 0x45, 0xf9, 0x8e, 0x3c, 0x86, 0x25, 0x21, 0x7d, 0x2e, 0x5f, - 0xf0, 0x64, 0xf8, 0x82, 0x45, 0xf4, 0x50, 0x19, 0x5d, 0x47, 0x91, 0x49, 0x06, 0xd9, 0x05, 0xc2, - 0xe2, 0x20, 0xca, 0x04, 0xbb, 0xa0, 0xc7, 0x05, 0xb7, 0x37, 0xbf, 0x6d, 0xed, 0xb4, 0xdc, 0x29, - 0x1c, 0xb2, 0x02, 0x8d, 0x88, 0x0d, 0x99, 0xec, 0x35, 0xb6, 0xad, 0x9d, 0xae, 0xab, 0x37, 0xce, - 0x2f, 0x61, 0xb9, 0x62, 0xff, 0x87, 0x3d, 0xff, 0xaf, 0x35, 0x68, 0x20, 0xa1, 0xf4, 0xb1, 0x35, - 0xf2, 0x31, 0xf9, 0x14, 0x3a, 0x4c, 0x78, 0x23, 0x47, 0xd4, 0xd0, 0xb6, 0x36, 0x13, 0xa5, 0xcf, - 0xc9, 0x23, 0x68, 0x06, 0x67, 0x59, 0x7c, 0x2e, 0x7a, 0xf5, 0xed, 0xfa, 0x4e, 0x7b, 0x6f, 0x79, - 0x74, 0x91, 0x7a, 0xe8, 0xbe, 0xe2, 0xb9, 0xb9, 0x08, 0xf9, 0x02, 0xc0, 0x97, 0x92, 0xb3, 0x93, - 0x4c, 0x52, 0x81, 0x2f, 0x6d, 0xef, 0xf5, 0x8c, 0x03, 0x99, 0xa0, 0x4f, 0x4b, 0xbe, 0x6b, 0xc8, - 0x92, 0x2f, 0xa1, 0x45, 0xaf, 0x24, 0x8d, 0x43, 0x1a, 0xf6, 0x1a, 0x78, 0xd1, 0xe6, 0xd8, 0x8b, - 0x76, 0x0f, 0x72, 0xbe, 0x7e, 0x5f, 0x29, 0xde, 0xff, 0x0a, 0xba, 0x15, 0x16, 0x59, 0x84, 0xfa, - 0x39, 0x2d, 0xa2, 0xaa, 0x96, 0xca, 0xb3, 0x17, 0x7e, 0x94, 0xe9, 0x04, 0xeb, 0xb8, 0x7a, 0xf3, - 0x8b, 0xda, 0x17, 0x96, 0xf3, 0x1c, 0xec, 0x17, 0x59, 0x14, 0x95, 0x07, 0x43, 0xc6, 0x8b, 0x83, - 0x21, 0xe3, 0x23, 0x2f, 0xd7, 0x6e, 0xf4, 0xf2, 0xdf, 0x2d, 0x58, 0x3a, 0xb8, 0xa0, 0xb1, 0x3c, - 0x4c, 0x24, 0x3b, 0x65, 0x81, 0x2f, 0x59, 0x12, 0x93, 0xc7, 0x60, 0x27, 0x51, 0xe8, 0xdd, 0x18, - 0xa6, 0x56, 0x12, 0xe5, 0x56, 0x3f, 0x06, 0x3b, 0xa6, 0x97, 0xde, 0x8d, 0xd7, 0xb5, 0x62, 0x7a, - 0xa9, 0xa5, 0x1f, 0x40, 0x37, 0xa4, 0x11, 0x95, 0xd4, 0x2b, 0xa3, 0xa3, 0x42, 0xd7, 0xd1, 0xc4, - 0x7d, 0x1d, 0x8e, 0xcf, 0xe0, 0x13, 0xa5, 0x32, 0xf5, 0x39, 0x8d, 0xa5, 0x97, 0xfa, 0xf2, 0x0c, - 0x63, 0x62, 0xbb, 0xdd, 0x98, 0x5e, 0x1e, 0x21, 0xf5, 0xc8, 0x97, 0x67, 0xce, 0xdf, 0x6a, 0x60, - 0x97, 0xc1, 0x24, 0xf7, 0x60, 0x41, 0x5d, 0xeb, 0xb1, 0x30, 0xf7, 0x44, 0x53, 0x6d, 0x5f, 0x85, - 0xaa, 0x2a, 0x92, 0xd3, 0x53, 0x41, 0x25, 0x9a, 0x57, 0x77, 0xf3, 0x9d, 0xca, 0x2c, 0xc1, 0xbe, - 0xd5, 0x85, 0x30, 0xef, 0xe2, 0x5a, 0x79, 0x7c, 0x28, 0xd9, 0x90, 0xe2, 0x85, 0x75, 0x57, 0x6f, - 0xc8, 0x32, 0x34, 0xa8, 0x27, 0xfd, 0x01, 0x66, 0xb8, 0xed, 0xce, 0xd3, 0x77, 0xfe, 0x80, 0xfc, - 0x08, 0xee, 0x88, 0x24, 0xe3, 0x01, 0xf5, 0x8a, 0x6b, 0x9b, 0xc8, 0xed, 0x68, 0xea, 0x0b, 0x7d, - 0xb9, 0x03, 0xf5, 0x53, 0x16, 0xf6, 0x16, 0xd0, 0x31, 0x8b, 0xd5, 0x24, 0x7c, 0x15, 0xba, 0x8a, - 0x49, 0x7e, 0x0a, 0x50, 0x6a, 0x0a, 0x7b, 0xad, 0x19, 0xa2, 0x76, 0xa1, 0x37, 0x24, 0x9b, 0x00, - 0x01, 0x4b, 0xcf, 0x28, 0xf7, 0x54, 0xc2, 0xd8, 0x98, 0x1c, 0xb6, 0xa6, 0xbc, 0xa6, 0xd7, 0x8a, - 0xcd, 0x84, 0x37, 0xf8, 0x96, 0xa5, 0x29, 0x0d, 0x7b, 0x80, 0x1e, 0xb6, 0x99, 0x78, 0xa9, 0x09, - 0xce, 0x6f, 0xa1, 0x99, 0x1b, 0xb7, 0x0e, 0xf6, 0x45, 0x12, 0x65, 0xc3, 0xd2, 0x69, 0x5d, 0xb7, - 0xa5, 0x09, 0xaf, 0x42, 0xb2, 0x06, 0x88, 0x92, 0x78, 0x45, 0x0d, 0x5d, 0x84, 0xfe, 0x55, 0x17, - 0xdc, 0x85, 0x66, 0x90, 0x24, 0xe7, 0x4c, 0xfb, 0x6e, 0xc1, 0xcd, 0x77, 0xce, 0x77, 0x75, 0xb8, - 0x53, 0x2d, 0x16, 0x75, 0x05, 0x6a, 0x41, 0x4f, 0x5b, 0xa8, 0x06, 0xd5, 0x1e, 0x57, 0xbc, 0x5d, - 0x33, 0xbd, 0x5d, 0x1c, 0x19, 0x26, 0xa1, 0xbe, 0xa0, 0xab, 0x8f, 0xbc, 0x4d, 0x42, 0xaa, 0x72, - 0x3d, 0x63, 0x21, 0x86, 0xa7, 0xeb, 0xaa, 0xa5, 0xa2, 0x0c, 0x58, 0x98, 0x83, 0x8f, 0x5a, 0xa2, - 0x79, 0x1c, 0xf5, 0x36, 0x75, 0xc0, 0xf5, 0x4e, 0x05, 0x7c, 0xa8, 0xa8, 0x0b, 0x3a, 0x8a, 0x6a, - 0x4d, 0xb6, 0xa1, 0xcd, 0x69, 0x1a, 0xe5, 0xb9, 0x8f, 0xce, 0xb7, 0x5d, 0x93, 0x44, 0xb6, 0x00, - 0x82, 0x24, 0x8a, 0x68, 0x80, 0x02, 0x36, 0x0a, 0x18, 0x14, 0x95, 0x77, 0x52, 0x46, 0x9e, 0xa0, - 0x01, 0xba, 0xba, 0xe1, 0x36, 0xa5, 0x8c, 0x8e, 0x69, 0xa0, 0xde, 0x91, 0x09, 0xca, 0x3d, 0x84, - 0xaf, 0x36, 0x9e, 0x6b, 0x29, 0x02, 0x82, 0xec, 0x26, 0xc0, 0x80, 0x27, 0x59, 0xaa, 0xb9, 0x9d, - 0xed, 0xba, 0x42, 0x72, 0xa4, 0x20, 0xfb, 0x21, 0xdc, 0x11, 0xd7, 0xc3, 0x88, 0xc5, 0xe7, 0x9e, - 0xf4, 0xf9, 0x80, 0xca, 0x5e, 0x57, 0x57, 0x40, 0x4e, 0x7d, 0x87, 0x44, 0xf5, 0xf6, 0x61, 0xf8, - 0xf3, 0xde, 0x1d, 0xcc, 0x00, 0xb5, 0x74, 0x52, 0x20, 0xfb, 0x9c, 0xfa, 0x92, 0x7e, 0x40, 0x1b, - 0xfb, 0x7e, 0x68, 0x41, 0x56, 0xa1, 0x99, 0x78, 0xf4, 0x2a, 0x88, 0xf2, 0xa2, 0x6d, 0x24, 0x07, - 0x57, 0x41, 0xe4, 0x3c, 0x82, 0xe5, 0xca, 0x8d, 0x39, 0xd0, 0xaf, 0x40, 0x83, 0x72, 0x9e, 0x14, - 0xb0, 0xa4, 0x37, 0xce, 0xef, 0x80, 0xbc, 0x4f, 0xc3, 0x8f, 0x61, 0x9e, 0xb3, 0x0a, 0xcb, 0x15, - 0xd5, 0xda, 0x0e, 0xe7, 0x3b, 0x0b, 0x56, 0x9e, 0xa6, 0x29, 0x8d, 0xc3, 0x77, 0xc9, 0x07, 0x5c, - 0xba, 0x09, 0x80, 0x6a, 0x3d, 0xa3, 0xc1, 0xdb, 0x48, 0xc1, 0xf8, 0x7c, 0x48, 0x7b, 0x71, 0xee, - 0xc1, 0xea, 0x98, 0x05, 0xb9, 0x6d, 0xff, 0xb2, 0x80, 0x3c, 0x47, 0xe4, 0xfb, 0x61, 0x43, 0x87, - 0xc2, 0x22, 0xd5, 0x10, 0x35, 0xb2, 0x86, 0xbe, 0xf4, 0xf3, 0x76, 0xdd, 0x61, 0x42, 0xeb, 0x7f, - 0xee, 0x4b, 0x3f, 0x6f, 0x9b, 0x9c, 0x06, 0x19, 0x57, 0x1d, 0x1c, 0x4b, 0x06, 0xdb, 0xa6, 0x5b, - 0x90, 0xc8, 0xe7, 0x70, 0x97, 0x0d, 0xe2, 0x84, 0xd3, 0x91, 0x98, 0xa7, 0xc3, 0xd8, 0x44, 0xe1, - 0x15, 0xcd, 0x2d, 0x0f, 0x1c, 0x60, 0x54, 0x1f, 0xc1, 0x72, 0xe5, 0x19, 0x37, 0xa6, 0xc0, 0x5f, - 0x2c, 0xe8, 0x3d, 0x95, 0xc9, 0x90, 0x05, 0x2e, 0x55, 0xc6, 0x57, 0x9e, 0xfe, 0x00, 0xba, 0xaa, - 0xf7, 0x8c, 0x3f, 0xbf, 0x93, 0x44, 0xe1, 0xa8, 0xb7, 0xaf, 0x81, 0x6a, 0x3f, 0x66, 0x64, 0x16, - 0x92, 0x28, 0xc4, 0xb8, 0x3c, 0x00, 0xd5, 0x23, 0x8c, 0xf3, 0x7a, 0xca, 0xe9, 0xc4, 0xf4, 0xb2, - 0x72, 0x5e, 0x09, 0xe1, 0x79, 0xdd, 0x58, 0x16, 0x62, 0x7a, 0xa9, 0xce, 0x3b, 0xeb, 0xb0, 0x36, - 0xc5, 0xb6, 0x3c, 0x5c, 0xff, 0xb6, 0x60, 0xf9, 0xa9, 0x10, 0x6c, 0x10, 0x7f, 0x83, 0x20, 0x59, - 0x18, 0xbd, 0x02, 0x8d, 0x20, 0xc9, 0x62, 0x89, 0xc6, 0x36, 0x5c, 0xbd, 0x19, 0xc3, 0x8d, 0xda, - 0x04, 0x6e, 0x8c, 0x21, 0x4f, 0x7d, 0x12, 0x79, 0x0c, 0x64, 0x99, 0xaf, 0x20, 0xcb, 0x7d, 0x68, - 0xab, 0x20, 0x7b, 0x01, 0x8d, 0x25, 0xe5, 0x79, 0x57, 0x02, 0x45, 0xda, 0x47, 0x8a, 0x12, 0x30, - 0xbb, 0xa7, 0x6e, 0x4c, 0x90, 0x8e, 0x5a, 0xe7, 0xff, 0x54, 0x55, 0x54, 0x9e, 0x92, 0xc7, 0x6c, - 0x66, 0x17, 0x55, 0xc0, 0xcb, 0xa3, 0xfc, 0x1d, 0x6a, 0xa9, 0x4a, 0x24, 0xcd, 0x4e, 0x22, 0x16, - 0x78, 0x8a, 0xa1, 0xed, 0xb7, 0x35, 0xe5, 0x3d, 0x8f, 0x46, 0x5e, 0x99, 0x37, 0xbd, 0x42, 0x60, - 0xde, 0xcf, 0xe4, 0x59, 0xd1, 0x49, 0xd5, 0x7a, 0xcc, 0x53, 0xcd, 0xdb, 0x3c, 0xb5, 0x30, 0xe9, - 0xa9, 0x32, 0xd3, 0x5a, 0x66, 0xa6, 0x7d, 0x0e, 0xcb, 0x7a, 0x14, 0xaf, 0x86, 0x6b, 0x13, 0xa0, - 0xec, 0x7a, 0xa2, 0x67, 0x69, 0xe8, 0x2d, 0xda, 0x9e, 0x70, 0x7e, 0x05, 0xf6, 0x9b, 0x44, 0xeb, - 0x15, 0xe4, 0x09, 0xd8, 0x51, 0xb1, 0x41, 0xd1, 0xf6, 0x1e, 0x19, 0x95, 0x7a, 0x21, 0xe7, 0x8e, - 0x84, 0x9c, 0xaf, 0xa0, 0x55, 0x90, 0x0b, 0x9f, 0x59, 0xb3, 0x7c, 0x56, 0x1b, 0xf3, 0x99, 0xf3, - 0x4f, 0x0b, 0x56, 0xaa, 0x26, 0xe7, 0x61, 0x79, 0x0f, 0xdd, 0xf2, 0x0a, 0x6f, 0xe8, 0xa7, 0xb9, - 0x2d, 0x4f, 0x4c, 0x5b, 0x26, 0x8f, 0x95, 0x06, 0x8a, 0xb7, 0x7e, 0xaa, 0x73, 0xb9, 0x13, 0x19, - 0xa4, 0xfe, 0x3b, 0x58, 0x9a, 0x10, 0x99, 0x32, 0x87, 0xfe, 0xc4, 0x9c, 0x43, 0x2b, 0x60, 0x57, - 0x9e, 0x36, 0x87, 0xd3, 0x2f, 0xe1, 0x9e, 0x86, 0x83, 0xfd, 0x32, 0x86, 0x85, 0xef, 0xab, 0xa1, - 0xb6, 0xc6, 0x43, 0xed, 0xf4, 0xa1, 0x37, 0x79, 0x34, 0x2f, 0xbf, 0x01, 0x2c, 0x1d, 0x4b, 0x5f, - 0x32, 0x21, 0x59, 0x50, 0x7e, 0x10, 0x8d, 0xe5, 0x86, 0x75, 0x5b, 0xff, 0x9e, 0xac, 0xc3, 0x45, - 0xa8, 0x4b, 0x59, 0xe4, 0xaf, 0x5a, 0xaa, 0x28, 0x10, 0xf3, 0xa6, 0x3c, 0x06, 0x1f, 0xe1, 0x2a, - 0x95, 0x0f, 0x32, 0x91, 0x7e, 0xa4, 0xe7, 0xa3, 0x79, 0x9c, 0x8f, 0x6c, 0xa4, 0xe0, 0x80, 0xa4, - 0x47, 0x88, 0x50, 0x73, 0x1b, 0x7a, 0x7a, 0x52, 0x04, 0x64, 0x6e, 0x02, 0x60, 0xa9, 0xea, 0x2a, - 0x6b, 0xea, 0xb3, 0x8a, 0xb2, 0xaf, 0x08, 0xce, 0x16, 0x6c, 0xbc, 0xa4, 0x52, 0x75, 0x23, 0xbe, - 0x9f, 0xc4, 0xa7, 0x6c, 0x90, 0x71, 0xdf, 0x08, 0x85, 0xf3, 0x1f, 0x0b, 0x36, 0x67, 0x08, 0xe4, - 0x0f, 0xee, 0xc1, 0xc2, 0xd0, 0x17, 0x92, 0xf2, 0xa2, 0x4a, 0x8a, 0xed, 0xb8, 0x2b, 0x6a, 0xb7, - 0xb9, 0xa2, 0x3e, 0xe1, 0x8a, 0x55, 0x68, 0x0e, 0xfd, 0x2b, 0x6f, 0x78, 0x92, 0x8f, 0x72, 0x8d, - 0xa1, 0x7f, 0xf5, 0xf6, 0x04, 0x91, 0x8d, 0x71, 0xef, 0x24, 0x0b, 0xce, 0xa9, 0x14, 0x25, 0xb2, - 0x31, 0xfe, 0x4c, 0x53, 0x70, 0xb6, 0xc3, 0x41, 0x17, 0x61, 0xa0, 0xe5, 0xe6, 0x3b, 0xe7, 0x12, - 0x7a, 0xc7, 0xd9, 0x89, 0x08, 0x38, 0x3b, 0xa1, 0x6f, 0xa9, 0xf4, 0x15, 0x18, 0x16, 0x39, 0x72, - 0x1f, 0xda, 0x41, 0xc4, 0x14, 0x1a, 0x1a, 0x5f, 0x92, 0xa0, 0x49, 0xd8, 0x35, 0x10, 0x2e, 0xe5, - 0x99, 0x57, 0xf9, 0x78, 0x06, 0x45, 0x3a, 0xd2, 0x1f, 0xd0, 0x6b, 0xd0, 0x12, 0x2c, 0x0e, 0xa8, - 0x17, 0xeb, 0x2f, 0x96, 0xba, 0xbb, 0x80, 0xfb, 0x43, 0xa1, 0xda, 0xd9, 0xda, 0x94, 0x9b, 0x73, - 0x17, 0xde, 0xdc, 0xca, 0x7f, 0x03, 0x84, 0x5e, 0xa0, 0x5d, 0xc6, 0xf7, 0x57, 0x5e, 0x64, 0xeb, - 0xc6, 0x98, 0x33, 0xfe, 0x89, 0xe6, 0x2e, 0xd1, 0x89, 0xaf, 0xb6, 0x65, 0x68, 0x48, 0x31, 0xb2, - 0x6f, 0x5e, 0x8a, 0x43, 0xe1, 0xf8, 0x0a, 0x8c, 0x06, 0xba, 0xac, 0x4b, 0x01, 0x6b, 0x24, 0x40, - 0x1e, 0x03, 0x49, 0x7d, 0x2e, 0x99, 0x52, 0xa1, 0x26, 0x7d, 0xef, 0xcc, 0x17, 0x67, 0x68, 0x41, - 0xc3, 0x5d, 0x2c, 0x39, 0xaf, 0xe9, 0xf5, 0xaf, 0x7d, 0x71, 0xa6, 0xc0, 0x1b, 0x87, 0x8b, 0x3a, - 0xce, 0x9b, 0xb8, 0x76, 0x5e, 0xc2, 0xca, 0x6b, 0x4a, 0xd3, 0xfd, 0x24, 0x8e, 0x69, 0x20, 0x69, - 0x58, 0x38, 0x7d, 0xda, 0x77, 0xfb, 0x3a, 0xd8, 0x03, 0x9e, 0x06, 0x5e, 0x9a, 0x70, 0xfd, 0x31, - 0xd6, 0x75, 0x5b, 0x8a, 0x70, 0x94, 0x70, 0xa9, 0xa6, 0xa4, 0x31, 0x45, 0xda, 0x87, 0x7b, 0xff, - 0xb5, 0xa1, 0x73, 0x4c, 0xfd, 0x4b, 0x4a, 0x43, 0x4c, 0x56, 0x32, 0x28, 0x40, 0xb2, 0xfa, 0x8b, - 0x85, 0x3c, 0x1c, 0x47, 0xc3, 0xa9, 0xff, 0x74, 0xfa, 0x9f, 0xdd, 0x26, 0x96, 0xe3, 0xcd, 0x1c, - 0x39, 0x84, 0xb6, 0xf1, 0x0f, 0x83, 0x6c, 0x18, 0x07, 0x27, 0x7e, 0xcd, 0xf4, 0x37, 0x67, 0x70, - 0x0b, 0x6d, 0x4f, 0x2c, 0xf2, 0x06, 0xda, 0xc6, 0xa8, 0x6c, 0xea, 0x9b, 0x9c, 0xd9, 0x4d, 0x7d, - 0x53, 0xe6, 0x6b, 0x67, 0x4e, 0x69, 0x33, 0x06, 0x5e, 0x53, 0xdb, 0xe4, 0x88, 0x6d, 0x6a, 0x9b, - 0x36, 0x25, 0xcf, 0x11, 0x17, 0xba, 0x95, 0x21, 0x95, 0x6c, 0x8d, 0x4e, 0x4c, 0x9b, 0x9f, 0xfb, - 0xf7, 0x67, 0xf2, 0x4d, 0x0b, 0x8d, 0xb9, 0xd0, 0xb4, 0x70, 0x72, 0xea, 0x35, 0x2d, 0x9c, 0x32, - 0x4c, 0x3a, 0x73, 0xe4, 0x0f, 0xb0, 0x34, 0x31, 0x9b, 0x11, 0xc7, 0xb0, 0x62, 0xc6, 0x50, 0xd9, - 0x7f, 0x70, 0xa3, 0x4c, 0xa9, 0xff, 0x6b, 0xe8, 0x98, 0x23, 0x11, 0x31, 0x0c, 0x9a, 0x32, 0xf5, - 0xf5, 0xb7, 0x66, 0xb1, 0x4d, 0x85, 0x66, 0x57, 0x36, 0x15, 0x4e, 0x99, 0x4b, 0x4c, 0x85, 0xd3, - 0x9a, 0xb9, 0x33, 0x47, 0x7e, 0x0f, 0x8b, 0xe3, 0xdd, 0x91, 0x7c, 0x3a, 0xee, 0xb6, 0x89, 0xa6, - 0xdb, 0x77, 0x6e, 0x12, 0x29, 0x95, 0xbf, 0x02, 0x18, 0x35, 0x3d, 0x62, 0xc0, 0xcf, 0x44, 0xd3, - 0xed, 0x6f, 0x4c, 0x67, 0x96, 0xaa, 0xfe, 0x04, 0xab, 0x53, 0x3b, 0x0b, 0x31, 0x4a, 0xef, 0xa6, - 0xde, 0xd4, 0xff, 0xf1, 0xad, 0x72, 0xe5, 0x5d, 0x7f, 0x84, 0xa5, 0x09, 0xf8, 0x35, 0xb3, 0x62, - 0x56, 0x57, 0x30, 0xb3, 0x62, 0x26, 0x7e, 0x63, 0xd5, 0x7e, 0x03, 0xdd, 0x0a, 0x30, 0x99, 0x95, - 0x31, 0x0d, 0xfa, 0xcc, 0xca, 0x98, 0x8a, 0x68, 0xce, 0xdc, 0x8e, 0xf5, 0xc4, 0x7a, 0xb6, 0x05, - 0x8b, 0x42, 0xc3, 0xda, 0xa9, 0xd8, 0xd5, 0xdd, 0xe8, 0x19, 0xe0, 0x5b, 0x8f, 0x78, 0x22, 0x93, - 0x93, 0x26, 0xfe, 0xab, 0xfe, 0xd9, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0xf8, 0xd4, 0xa1, 0xe3, - 0xba, 0x16, 0x00, 0x00, + // 2142 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x59, 0x5f, 0x6f, 0xdb, 0xc8, + 0x11, 0x37, 0x25, 0x4b, 0x16, 0x47, 0x52, 0xce, 0x5e, 0x3b, 0x89, 0xa2, 0xc4, 0x89, 0x8f, 0x69, + 0xee, 0x5c, 0x24, 0x70, 0x53, 0xf7, 0x0a, 0xdc, 0xf5, 0xda, 0x07, 0xc7, 0x71, 0xae, 0x69, 0x12, + 0x9f, 0x41, 0x27, 0x87, 0x2b, 0x0a, 0x94, 0xa5, 0xc9, 0xb5, 0xbc, 0x35, 0x45, 0xb2, 0xbb, 0x4b, + 0xff, 0xb9, 0xa7, 0xfb, 0x1c, 0x05, 0xfa, 0xda, 0x4f, 0xd0, 0xc7, 0xa2, 0x2f, 0x45, 0x81, 0x02, + 0x45, 0xbf, 0x44, 0x3f, 0x49, 0xb1, 0xb3, 0x24, 0xb5, 0x14, 0x25, 0xfb, 0x82, 0xc3, 0xbd, 0x71, + 0x67, 0x66, 0x67, 0x67, 0xe7, 0xcf, 0x6f, 0x66, 0x25, 0xe8, 0x1e, 0xb3, 0x88, 0xf2, 0xad, 0x94, + 0x27, 0x32, 0x21, 0x1d, 0x5c, 0x78, 0xe9, 0x91, 0xf3, 0x25, 0xdc, 0x7d, 0x9d, 0x24, 0xa7, 0x59, + 0xfa, 0x9c, 0x71, 0x1a, 0xc8, 0x84, 0x5f, 0xee, 0xc5, 0x92, 0x5f, 0xba, 0xf4, 0x4f, 0x19, 0x15, + 0x92, 0xdc, 0x03, 0x3b, 0x2c, 0x18, 0x03, 0x6b, 0xc3, 0xda, 0xb4, 0xdd, 0x09, 0x81, 0x10, 0x58, + 0x8c, 0xfd, 0x31, 0x1d, 0x34, 0x90, 0x81, 0xdf, 0xce, 0x1e, 0xdc, 0x9b, 0xad, 0x50, 0xa4, 0x49, + 0x2c, 0x28, 0x79, 0x04, 0x2d, 0xaa, 0x08, 0xa8, 0xad, 0xbb, 0xfd, 0xc1, 0x56, 0x61, 0xca, 0x96, + 0x96, 0xd3, 0x5c, 0xe7, 0x1f, 0x16, 0x90, 0xd7, 0x4c, 0x48, 0x45, 0x64, 0x54, 0x7c, 0x37, 0x7b, + 0x6e, 0x41, 0x3b, 0xe5, 0xf4, 0x98, 0x5d, 0xe4, 0x16, 0xe5, 0x2b, 0xf2, 0x04, 0x56, 0x84, 0xf4, + 0xb9, 0x7c, 0xc1, 0x93, 0xf1, 0x0b, 0x16, 0xd1, 0x7d, 0x65, 0x74, 0x13, 0x45, 0xea, 0x0c, 0xb2, + 0x05, 0x84, 0xc5, 0x41, 0x94, 0x09, 0x76, 0x46, 0x0f, 0x0b, 0xee, 0x60, 0x71, 0xc3, 0xda, 0xec, + 0xb8, 0x33, 0x38, 0x64, 0x0d, 0x5a, 0x11, 0x1b, 0x33, 0x39, 0x68, 0x6d, 0x58, 0x9b, 0x7d, 0x57, + 0x2f, 0x9c, 0x5f, 0xc2, 0x6a, 0xc5, 0xfe, 0xf7, 0xbb, 0xfe, 0x5f, 0x1a, 0xd0, 0x42, 0x42, 0xe9, + 0x63, 0x6b, 0xe2, 0x63, 0xf2, 0x21, 0xf4, 0x98, 0xf0, 0x26, 0x8e, 0x68, 0xa0, 0x6d, 0x5d, 0x26, + 0x4a, 0x9f, 0x93, 0xc7, 0xd0, 0x0e, 0x4e, 0xb2, 0xf8, 0x54, 0x0c, 0x9a, 0x1b, 0xcd, 0xcd, 0xee, + 0xf6, 0xea, 0xe4, 0x20, 0x75, 0xd1, 0x5d, 0xc5, 0x73, 0x73, 0x11, 0xf2, 0x29, 0x80, 0x2f, 0x25, + 0x67, 0x47, 0x99, 0xa4, 0x02, 0x6f, 0xda, 0xdd, 0x1e, 0x18, 0x1b, 0x32, 0x41, 0x77, 0x4a, 0xbe, + 0x6b, 0xc8, 0x92, 0xcf, 0xa0, 0x43, 0x2f, 0x24, 0x8d, 0x43, 0x1a, 0x0e, 0x5a, 0x78, 0xd0, 0xfa, + 0xd4, 0x8d, 0xb6, 0xf6, 0x72, 0xbe, 0xbe, 0x5f, 0x29, 0x3e, 0xfc, 0x1c, 0xfa, 0x15, 0x16, 0x59, + 0x86, 0xe6, 0x29, 0x2d, 0xa2, 0xaa, 0x3e, 0x95, 0x67, 0xcf, 0xfc, 0x28, 0xd3, 0x09, 0xd6, 0x73, + 0xf5, 0xe2, 0x17, 0x8d, 0x4f, 0x2d, 0xe7, 0x39, 0xd8, 0x2f, 0xb2, 0x28, 0x2a, 0x37, 0x86, 0x8c, + 0x17, 0x1b, 0x43, 0xc6, 0x27, 0x5e, 0x6e, 0x5c, 0xe9, 0xe5, 0xbf, 0x5b, 0xb0, 0xb2, 0x77, 0x46, + 0x63, 0xb9, 0x9f, 0x48, 0x76, 0xcc, 0x02, 0x5f, 0xb2, 0x24, 0x26, 0x4f, 0xc0, 0x4e, 0xa2, 0xd0, + 0xbb, 0x32, 0x4c, 0x9d, 0x24, 0xca, 0xad, 0x7e, 0x02, 0x76, 0x4c, 0xcf, 0xbd, 0x2b, 0x8f, 0xeb, + 0xc4, 0xf4, 0x5c, 0x4b, 0x3f, 0x84, 0x7e, 0x48, 0x23, 0x2a, 0xa9, 0x57, 0x46, 0x47, 0x85, 0xae, + 0xa7, 0x89, 0xbb, 0x3a, 0x1c, 0x1f, 0xc1, 0x07, 0x4a, 0x65, 0xea, 0x73, 0x1a, 0x4b, 0x2f, 0xf5, + 0xe5, 0x09, 0xc6, 0xc4, 0x76, 0xfb, 0x31, 0x3d, 0x3f, 0x40, 0xea, 0x81, 0x2f, 0x4f, 0x9c, 0xbf, + 0x35, 0xc0, 0x2e, 0x83, 0x49, 0x6e, 0xc3, 0x92, 0x3a, 0xd6, 0x63, 0x61, 0xee, 0x89, 0xb6, 0x5a, + 0xbe, 0x0c, 0x55, 0x55, 0x24, 0xc7, 0xc7, 0x82, 0x4a, 0x34, 0xaf, 0xe9, 0xe6, 0x2b, 0x95, 0x59, + 0x82, 0x7d, 0xa3, 0x0b, 0x61, 0xd1, 0xc5, 0x6f, 0xe5, 0xf1, 0xb1, 0x64, 0x63, 0x8a, 0x07, 0x36, + 0x5d, 0xbd, 0x20, 0xab, 0xd0, 0xa2, 0x9e, 0xf4, 0x47, 0x98, 0xe1, 0xb6, 0xbb, 0x48, 0xdf, 0xfa, + 0x23, 0xf2, 0x23, 0xb8, 0x21, 0x92, 0x8c, 0x07, 0xd4, 0x2b, 0x8e, 0x6d, 0x23, 0xb7, 0xa7, 0xa9, + 0x2f, 0xf4, 0xe1, 0x0e, 0x34, 0x8f, 0x59, 0x38, 0x58, 0x42, 0xc7, 0x2c, 0x57, 0x93, 0xf0, 0x65, + 0xe8, 0x2a, 0x26, 0xf9, 0x09, 0x40, 0xa9, 0x29, 0x1c, 0x74, 0xe6, 0x88, 0xda, 0x85, 0xde, 0x90, + 0xac, 0x03, 0x04, 0x2c, 0x3d, 0xa1, 0xdc, 0x53, 0x09, 0x63, 0x63, 0x72, 0xd8, 0x9a, 0xf2, 0x8a, + 0x5e, 0x2a, 0x36, 0x13, 0xde, 0xe8, 0x1b, 0x96, 0xa6, 0x34, 0x1c, 0x00, 0x7a, 0xd8, 0x66, 0xe2, + 0x0b, 0x4d, 0x70, 0xbe, 0x86, 0x76, 0x6e, 0xdc, 0x5d, 0xb0, 0xcf, 0x92, 0x28, 0x1b, 0x97, 0x4e, + 0xeb, 0xbb, 0x1d, 0x4d, 0x78, 0x19, 0x92, 0x3b, 0x80, 0x28, 0x89, 0x47, 0x34, 0xd0, 0x45, 0xe8, + 0x5f, 0x75, 0xc0, 0x2d, 0x68, 0x07, 0x49, 0x72, 0xca, 0xb4, 0xef, 0x96, 0xdc, 0x7c, 0xe5, 0x7c, + 0xdb, 0x84, 0x1b, 0xd5, 0x62, 0x51, 0x47, 0xa0, 0x16, 0xf4, 0xb4, 0x85, 0x6a, 0x50, 0xed, 0x61, + 0xc5, 0xdb, 0x0d, 0xd3, 0xdb, 0xc5, 0x96, 0x71, 0x12, 0xea, 0x03, 0xfa, 0x7a, 0xcb, 0x9b, 0x24, + 0xa4, 0x2a, 0xd7, 0x33, 0x16, 0x62, 0x78, 0xfa, 0xae, 0xfa, 0x54, 0x94, 0x11, 0x0b, 0x73, 0xf0, + 0x51, 0x9f, 0x68, 0x1e, 0x47, 0xbd, 0x6d, 0x1d, 0x70, 0xbd, 0x52, 0x01, 0x1f, 0x2b, 0xea, 0x92, + 0x8e, 0xa2, 0xfa, 0x26, 0x1b, 0xd0, 0xe5, 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, 0xe5, 0xb6, 0xa5, 0x8c, 0x0e, 0x69, 0xa0, 0xee, 0x91, 0x09, + 0xca, 0x3d, 0x84, 0xaf, 0x2e, 0xee, 0xeb, 0x28, 0x02, 0x82, 0xec, 0x3a, 0xc0, 0x88, 0x27, 0x59, + 0xaa, 0xb9, 0xbd, 0x8d, 0xa6, 0x42, 0x72, 0xa4, 0x20, 0xfb, 0x11, 0xdc, 0x10, 0x97, 0xe3, 0x88, + 0xc5, 0xa7, 0x9e, 0xf4, 0xf9, 0x88, 0xca, 0x41, 0x5f, 0x57, 0x40, 0x4e, 0x7d, 0x8b, 0x44, 0x75, + 0xf7, 0x71, 0xf8, 0xf3, 0xc1, 0x0d, 0xcc, 0x00, 0xf5, 0xe9, 0xa4, 0x40, 0x76, 0x39, 0xf5, 0x25, + 0x7d, 0x8f, 0x36, 0xf6, 0xdd, 0xd0, 0x82, 0xdc, 0x84, 0x76, 0xe2, 0xd1, 0x8b, 0x20, 0xca, 0x8b, + 0xb6, 0x95, 0xec, 0x5d, 0x04, 0x91, 0xf3, 0x18, 0x56, 0x2b, 0x27, 0xe6, 0x40, 0xbf, 0x06, 0x2d, + 0xca, 0x79, 0x52, 0xc0, 0x92, 0x5e, 0x38, 0xbf, 0x05, 0xf2, 0x2e, 0x0d, 0x7f, 0x08, 0xf3, 0x9c, + 0x9b, 0xb0, 0x5a, 0x51, 0xad, 0xed, 0x70, 0xbe, 0xb5, 0x60, 0x6d, 0x27, 0x4d, 0x69, 0x1c, 0xbe, + 0x4d, 0xde, 0xe3, 0xd0, 0x75, 0x00, 0x54, 0xeb, 0x19, 0x0d, 0xde, 0x46, 0x0a, 0xc6, 0xe7, 0x7d, + 0xda, 0x8b, 0x73, 0x1b, 0x6e, 0x4e, 0x59, 0x90, 0xdb, 0xf6, 0x2f, 0x0b, 0xc8, 0x73, 0x44, 0xbe, + 0xef, 0x37, 0x74, 0x28, 0x2c, 0x52, 0x0d, 0x51, 0x23, 0x6b, 0xe8, 0x4b, 0x3f, 0x6f, 0xd7, 0x3d, + 0x26, 0xb4, 0xfe, 0xe7, 0xbe, 0xf4, 0xf3, 0xb6, 0xc9, 0x69, 0x90, 0x71, 0xd5, 0xc1, 0xb1, 0x64, + 0xb0, 0x6d, 0xba, 0x05, 0x89, 0x7c, 0x02, 0xb7, 0xd8, 0x28, 0x4e, 0x38, 0x9d, 0x88, 0x79, 0x3a, + 0x8c, 0x6d, 0x14, 0x5e, 0xd3, 0xdc, 0x72, 0xc3, 0x1e, 0x46, 0xf5, 0x31, 0xac, 0x56, 0xae, 0x71, + 0x65, 0x0a, 0xfc, 0xd9, 0x82, 0xc1, 0x8e, 0x4c, 0xc6, 0x2c, 0x70, 0xa9, 0x32, 0xbe, 0x72, 0xf5, + 0x87, 0xd0, 0x57, 0xbd, 0x67, 0xfa, 0xfa, 0xbd, 0x24, 0x0a, 0x27, 0xbd, 0xfd, 0x0e, 0xa8, 0xf6, + 0x63, 0x46, 0x66, 0x29, 0x89, 0x42, 0x8c, 0xcb, 0x43, 0x50, 0x3d, 0xc2, 0xd8, 0xaf, 0xa7, 0x9c, + 0x5e, 0x4c, 0xcf, 0x2b, 0xfb, 0x95, 0x10, 0xee, 0xd7, 0x8d, 0x65, 0x29, 0xa6, 0xe7, 0x6a, 0xbf, + 0x73, 0x17, 0xee, 0xcc, 0xb0, 0x2d, 0x0f, 0xd7, 0xbf, 0x2d, 0x58, 0xdd, 0x11, 0x82, 0x8d, 0xe2, + 0xaf, 0x10, 0x24, 0x0b, 0xa3, 0xd7, 0xa0, 0x15, 0x24, 0x59, 0x2c, 0xd1, 0xd8, 0x96, 0xab, 0x17, + 0x53, 0xb8, 0xd1, 0xa8, 0xe1, 0xc6, 0x14, 0xf2, 0x34, 0xeb, 0xc8, 0x63, 0x20, 0xcb, 0x62, 0x05, + 0x59, 0x1e, 0x40, 0x57, 0x05, 0xd9, 0x0b, 0x68, 0x2c, 0x29, 0xcf, 0xbb, 0x12, 0x28, 0xd2, 0x2e, + 0x52, 0x94, 0x80, 0xd9, 0x3d, 0x75, 0x63, 0x82, 0x74, 0xd2, 0x3a, 0xff, 0xa7, 0xaa, 0xa2, 0x72, + 0x95, 0x3c, 0x66, 0x73, 0xbb, 0xa8, 0x02, 0x5e, 0x1e, 0xe5, 0xf7, 0x50, 0x9f, 0xaa, 0x44, 0xd2, + 0xec, 0x28, 0x62, 0x81, 0xa7, 0x18, 0xda, 0x7e, 0x5b, 0x53, 0xde, 0xf1, 0x68, 0xe2, 0x95, 0x45, + 0xd3, 0x2b, 0x04, 0x16, 0xfd, 0x4c, 0x9e, 0x14, 0x9d, 0x54, 0x7d, 0x4f, 0x79, 0xaa, 0x7d, 0x9d, + 0xa7, 0x96, 0xea, 0x9e, 0x2a, 0x33, 0xad, 0x63, 0x66, 0xda, 0x27, 0xb0, 0xaa, 0x47, 0xf1, 0x6a, + 0xb8, 0xd6, 0x01, 0xca, 0xae, 0x27, 0x06, 0x96, 0x86, 0xde, 0xa2, 0xed, 0x09, 0xe7, 0x57, 0x60, + 0xbf, 0x4e, 0xb4, 0x5e, 0x41, 0x9e, 0x82, 0x1d, 0x15, 0x0b, 0x14, 0xed, 0x6e, 0x93, 0x49, 0xa9, + 0x17, 0x72, 0xee, 0x44, 0xc8, 0xf9, 0x1c, 0x3a, 0x05, 0xb9, 0xf0, 0x99, 0x35, 0xcf, 0x67, 0x8d, + 0x29, 0x9f, 0x39, 0xff, 0xb4, 0x60, 0xad, 0x6a, 0x72, 0x1e, 0x96, 0x77, 0xd0, 0x2f, 0x8f, 0xf0, + 0xc6, 0x7e, 0x9a, 0xdb, 0xf2, 0xd4, 0xb4, 0xa5, 0xbe, 0xad, 0x34, 0x50, 0xbc, 0xf1, 0x53, 0x9d, + 0xcb, 0xbd, 0xc8, 0x20, 0x0d, 0xdf, 0xc2, 0x4a, 0x4d, 0x64, 0xc6, 0x1c, 0xfa, 0x63, 0x73, 0x0e, + 0xad, 0x80, 0x5d, 0xb9, 0xdb, 0x1c, 0x4e, 0x3f, 0x83, 0xdb, 0x1a, 0x0e, 0x76, 0xcb, 0x18, 0x16, + 0xbe, 0xaf, 0x86, 0xda, 0x9a, 0x0e, 0xb5, 0x33, 0x84, 0x41, 0x7d, 0x6b, 0x5e, 0x7e, 0x23, 0x58, + 0x39, 0x94, 0xbe, 0x64, 0x42, 0xb2, 0xa0, 0x7c, 0x10, 0x4d, 0xe5, 0x86, 0x75, 0x5d, 0xff, 0xae, + 0xd7, 0xe1, 0x32, 0x34, 0xa5, 0x2c, 0xf2, 0x57, 0x7d, 0xaa, 0x28, 0x10, 0xf3, 0xa4, 0x3c, 0x06, + 0x3f, 0xc0, 0x51, 0x2a, 0x1f, 0x64, 0x22, 0xfd, 0x48, 0xcf, 0x47, 0x8b, 0x38, 0x1f, 0xd9, 0x48, + 0xc1, 0x01, 0x49, 0x8f, 0x10, 0xa1, 0xe6, 0xb6, 0xf4, 0xf4, 0xa4, 0x08, 0xc8, 0x5c, 0x07, 0xc0, + 0x52, 0xd5, 0x55, 0xd6, 0xd6, 0x7b, 0x15, 0x65, 0x57, 0x11, 0x9c, 0xfb, 0x70, 0xef, 0x0b, 0x2a, + 0x55, 0x37, 0xe2, 0xbb, 0x49, 0x7c, 0xcc, 0x46, 0x19, 0xf7, 0x8d, 0x50, 0x38, 0xff, 0xb1, 0x60, + 0x7d, 0x8e, 0x40, 0x7e, 0xe1, 0x01, 0x2c, 0x8d, 0x7d, 0x21, 0x29, 0x2f, 0xaa, 0xa4, 0x58, 0x4e, + 0xbb, 0xa2, 0x71, 0x9d, 0x2b, 0x9a, 0x35, 0x57, 0xdc, 0x84, 0xf6, 0xd8, 0xbf, 0xf0, 0xc6, 0x47, + 0xf9, 0x28, 0xd7, 0x1a, 0xfb, 0x17, 0x6f, 0x8e, 0x10, 0xd9, 0x18, 0xf7, 0x8e, 0xb2, 0xe0, 0x94, + 0x4a, 0x51, 0x22, 0x1b, 0xe3, 0xcf, 0x34, 0x05, 0x67, 0x3b, 0x1c, 0x74, 0x11, 0x06, 0x3a, 0x6e, + 0xbe, 0x72, 0xce, 0x61, 0x70, 0x98, 0x1d, 0x89, 0x80, 0xb3, 0x23, 0xfa, 0x86, 0x4a, 0x5f, 0x81, + 0x61, 0x91, 0x23, 0x0f, 0xa0, 0x1b, 0x44, 0x4c, 0xa1, 0xa1, 0xf1, 0x92, 0x04, 0x4d, 0xc2, 0xae, + 0x81, 0x70, 0x29, 0x4f, 0xbc, 0xca, 0xe3, 0x19, 0x14, 0xe9, 0x40, 0x3f, 0xa0, 0xef, 0x40, 0x47, + 0xb0, 0x38, 0xa0, 0x5e, 0xac, 0x5f, 0x2c, 0x4d, 0x77, 0x09, 0xd7, 0xfb, 0x42, 0xb5, 0xb3, 0x3b, + 0x33, 0x4e, 0xce, 0x5d, 0x78, 0x75, 0x2b, 0xff, 0x0d, 0x10, 0x7a, 0x86, 0x76, 0x19, 0xef, 0xaf, + 0xbc, 0xc8, 0xee, 0x1a, 0x63, 0xce, 0xf4, 0x13, 0xcd, 0x5d, 0xa1, 0xb5, 0x57, 0xdb, 0x2a, 0xb4, + 0xa4, 0x98, 0xd8, 0xb7, 0x28, 0xc5, 0xbe, 0x70, 0x7c, 0x05, 0x46, 0x23, 0x5d, 0xd6, 0xa5, 0x80, + 0x35, 0x11, 0x20, 0x4f, 0x80, 0xa4, 0x3e, 0x97, 0x4c, 0xa9, 0x50, 0x93, 0xbe, 0x77, 0xe2, 0x8b, + 0x13, 0xb4, 0xa0, 0xe5, 0x2e, 0x97, 0x9c, 0x57, 0xf4, 0xf2, 0xd7, 0xbe, 0x38, 0x51, 0xe0, 0x8d, + 0xc3, 0x45, 0x13, 0xe7, 0x4d, 0xfc, 0x76, 0x28, 0xac, 0xbd, 0xa2, 0x34, 0xdd, 0x4d, 0xe2, 0x98, + 0x06, 0x92, 0x86, 0x85, 0xd3, 0x67, 0xbd, 0xdb, 0xef, 0x82, 0x3d, 0xe2, 0x69, 0xe0, 0xa5, 0x09, + 0xd7, 0x8f, 0xb1, 0xbe, 0xdb, 0x51, 0x84, 0x83, 0x84, 0xe3, 0xd4, 0xc3, 0xa9, 0x7e, 0xe3, 0xe8, + 0xa9, 0xca, 0x76, 0x27, 0x04, 0x35, 0x43, 0x4d, 0x1d, 0x93, 0xa3, 0xc2, 0x4f, 0x15, 0xc8, 0x07, + 0xbe, 0xa4, 0xcf, 0x78, 0x72, 0x4a, 0x79, 0x71, 0xfc, 0x10, 0x3a, 0xc5, 0xe6, 0xdc, 0x84, 0x72, + 0xed, 0xfc, 0x17, 0x51, 0xd6, 0xdc, 0x33, 0x19, 0x58, 0x8e, 0x93, 0x2c, 0xd6, 0xad, 0xaf, 0xe3, + 0xea, 0x05, 0xd9, 0x33, 0x0d, 0x6b, 0x20, 0xee, 0x7e, 0x3c, 0x85, 0x80, 0x53, 0x8a, 0xb6, 0xdc, + 0x5c, 0xde, 0xb8, 0xc1, 0xf0, 0x6b, 0xe8, 0x14, 0x64, 0x35, 0xde, 0xa3, 0x23, 0xfc, 0x30, 0xe4, + 0x54, 0x08, 0x2a, 0x72, 0x1b, 0xfb, 0x8a, 0xba, 0x53, 0x10, 0x95, 0x58, 0xb1, 0x3f, 0xaf, 0x72, + 0x1d, 0x99, 0x7e, 0x41, 0xc5, 0x4a, 0xdf, 0xfe, 0x2b, 0x40, 0xef, 0x90, 0xfa, 0xe7, 0x94, 0x86, + 0x58, 0xcd, 0x64, 0x54, 0x74, 0x91, 0xea, 0x6f, 0x50, 0xe4, 0xd1, 0x74, 0xbb, 0x98, 0xf9, 0xa3, + 0xd7, 0xf0, 0xa3, 0xeb, 0xc4, 0x72, 0xd7, 0x2f, 0x90, 0x7d, 0xe8, 0x1a, 0x3f, 0xf2, 0x90, 0x7b, + 0xc6, 0xc6, 0xda, 0x6f, 0x57, 0xc3, 0xf5, 0x39, 0xdc, 0x42, 0xdb, 0x53, 0x8b, 0xbc, 0x86, 0xae, + 0xf1, 0x96, 0x30, 0xf5, 0xd5, 0x1f, 0x35, 0xa6, 0xbe, 0x19, 0x0f, 0x10, 0x67, 0x41, 0x69, 0x33, + 0x5e, 0x04, 0xa6, 0xb6, 0xfa, 0x1b, 0xc4, 0xd4, 0x36, 0xeb, 0x19, 0xb1, 0x40, 0x5c, 0xe8, 0x57, + 0xa6, 0x78, 0x72, 0x7f, 0xb2, 0x63, 0xd6, 0x03, 0x63, 0xf8, 0x60, 0x2e, 0xdf, 0xb4, 0xd0, 0x18, + 0x9c, 0x4d, 0x0b, 0xeb, 0xcf, 0x02, 0xd3, 0xc2, 0x19, 0xd3, 0xb6, 0xb3, 0x40, 0x7e, 0x0f, 0x2b, + 0xb5, 0xe1, 0x95, 0x38, 0x86, 0x15, 0x73, 0xa6, 0xee, 0xe1, 0xc3, 0x2b, 0x65, 0x4a, 0xfd, 0x5f, + 0x42, 0xcf, 0x9c, 0x19, 0x89, 0x61, 0xd0, 0x8c, 0xb1, 0x78, 0x78, 0x7f, 0x1e, 0xdb, 0x54, 0x68, + 0x8e, 0x2d, 0xa6, 0xc2, 0x19, 0x83, 0x9b, 0xa9, 0x70, 0xd6, 0xb4, 0xe3, 0x2c, 0x90, 0xdf, 0xc1, + 0xf2, 0xf4, 0xf8, 0x40, 0x3e, 0x9c, 0x76, 0x5b, 0x6d, 0x2a, 0x19, 0x3a, 0x57, 0x89, 0x94, 0xca, + 0x5f, 0x02, 0x4c, 0xa6, 0x02, 0x62, 0xe0, 0x73, 0x6d, 0x2a, 0x19, 0xde, 0x9b, 0xcd, 0x2c, 0x55, + 0xfd, 0x11, 0x6e, 0xce, 0x6c, 0xbd, 0xc4, 0x28, 0xbd, 0xab, 0x9a, 0xf7, 0xf0, 0xe3, 0x6b, 0xe5, + 0xca, 0xb3, 0xfe, 0x00, 0x2b, 0xb5, 0xfe, 0x64, 0x66, 0xc5, 0xbc, 0xb6, 0x69, 0x66, 0xc5, 0xdc, + 0x06, 0x87, 0x55, 0xfb, 0x15, 0xf4, 0x2b, 0xd8, 0x6c, 0x56, 0xc6, 0xac, 0xde, 0x60, 0x56, 0xc6, + 0x6c, 0x50, 0x5f, 0xd8, 0xb4, 0x9e, 0x5a, 0x3a, 0x3d, 0x26, 0xe8, 0x5a, 0x4d, 0x8f, 0x1a, 0xe4, + 0x57, 0xd3, 0xa3, 0x0e, 0xca, 0xce, 0xc2, 0xb3, 0xfb, 0xb0, 0x2c, 0x34, 0x4e, 0x1e, 0x8b, 0x2d, + 0xdd, 0xff, 0x9f, 0x01, 0x3a, 0xef, 0x80, 0x27, 0x32, 0x39, 0x6a, 0xe3, 0xbf, 0x03, 0x3f, 0xfb, + 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x32, 0x17, 0xbf, 0x34, 0x2c, 0x18, 0x00, 0x00, } diff --git a/weed/pb/messaging.proto b/weed/pb/messaging.proto index 56f60b212..689c22d29 100644 --- a/weed/pb/messaging.proto +++ b/weed/pb/messaging.proto @@ -15,6 +15,9 @@ service SeaweedMessaging { rpc Publish (stream PublishRequest) returns (stream PublishResponse) { } + rpc DeleteTopic (DeleteTopicRequest) returns (DeleteTopicResponse) { + } + rpc ConfigureTopic (ConfigureTopicRequest) returns (ConfigureTopicResponse) { } @@ -47,6 +50,7 @@ message SubscriberMessage { int64 message_id = 1; } AckMessage ack = 2; + bool is_close = 3; } message Message { @@ -54,11 +58,11 @@ message Message { bytes key = 2; // Message key bytes value = 3; // Message payload map headers = 4; // Message headers + bool is_close = 5; } message BrokerMessage { Message data = 1; - bool is_close = 2; } message PublishRequest { @@ -80,6 +84,14 @@ message PublishResponse { string new_broker = 1; } RedirectMessage redirect = 2; + bool is_closed = 3; +} + +message DeleteTopicRequest { + string namespace = 1; + string topic = 2; +} +message DeleteTopicResponse { } message ConfigureTopicRequest { diff --git a/weed/pb/messaging_pb/messaging.pb.go b/weed/pb/messaging_pb/messaging.pb.go index 809fd3ded..f42e2c2db 100644 --- a/weed/pb/messaging_pb/messaging.pb.go +++ b/weed/pb/messaging_pb/messaging.pb.go @@ -14,6 +14,8 @@ It has these top-level messages: BrokerMessage PublishRequest PublishResponse + DeleteTopicRequest + DeleteTopicResponse ConfigureTopicRequest ConfigureTopicResponse GetTopicConfigurationRequest @@ -93,12 +95,13 @@ func (x TopicConfiguration_Partitioning) String() string { return proto.EnumName(TopicConfiguration_Partitioning_name, int32(x)) } func (TopicConfiguration_Partitioning) EnumDescriptor() ([]byte, []int) { - return fileDescriptor0, []int{11, 0} + return fileDescriptor0, []int{13, 0} } type SubscriberMessage struct { - Init *SubscriberMessage_InitMessage `protobuf:"bytes,1,opt,name=init" json:"init,omitempty"` - Ack *SubscriberMessage_AckMessage `protobuf:"bytes,2,opt,name=ack" json:"ack,omitempty"` + Init *SubscriberMessage_InitMessage `protobuf:"bytes,1,opt,name=init" json:"init,omitempty"` + Ack *SubscriberMessage_AckMessage `protobuf:"bytes,2,opt,name=ack" json:"ack,omitempty"` + IsClose bool `protobuf:"varint,3,opt,name=is_close,json=isClose" json:"is_close,omitempty"` } func (m *SubscriberMessage) Reset() { *m = SubscriberMessage{} } @@ -120,6 +123,13 @@ func (m *SubscriberMessage) GetAck() *SubscriberMessage_AckMessage { return nil } +func (m *SubscriberMessage) GetIsClose() bool { + if m != nil { + return m.IsClose + } + return false +} + type SubscriberMessage_InitMessage struct { Namespace string `protobuf:"bytes,1,opt,name=namespace" json:"namespace,omitempty"` Topic string `protobuf:"bytes,2,opt,name=topic" json:"topic,omitempty"` @@ -199,6 +209,7 @@ type Message struct { Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` Value []byte `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` Headers map[string][]byte `protobuf:"bytes,4,rep,name=headers" json:"headers,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value,proto3"` + IsClose bool `protobuf:"varint,5,opt,name=is_close,json=isClose" json:"is_close,omitempty"` } func (m *Message) Reset() { *m = Message{} } @@ -234,9 +245,15 @@ func (m *Message) GetHeaders() map[string][]byte { return nil } +func (m *Message) GetIsClose() bool { + if m != nil { + return m.IsClose + } + return false +} + type BrokerMessage struct { - Data *Message `protobuf:"bytes,1,opt,name=data" json:"data,omitempty"` - IsClose bool `protobuf:"varint,2,opt,name=is_close,json=isClose" json:"is_close,omitempty"` + Data *Message `protobuf:"bytes,1,opt,name=data" json:"data,omitempty"` } func (m *BrokerMessage) Reset() { *m = BrokerMessage{} } @@ -251,13 +268,6 @@ func (m *BrokerMessage) GetData() *Message { return nil } -func (m *BrokerMessage) GetIsClose() bool { - if m != nil { - return m.IsClose - } - return false -} - type PublishRequest struct { Init *PublishRequest_InitMessage `protobuf:"bytes,1,opt,name=init" json:"init,omitempty"` Data *Message `protobuf:"bytes,2,opt,name=data" json:"data,omitempty"` @@ -317,6 +327,7 @@ func (m *PublishRequest_InitMessage) GetPartition() int32 { type PublishResponse struct { Config *PublishResponse_ConfigMessage `protobuf:"bytes,1,opt,name=config" json:"config,omitempty"` Redirect *PublishResponse_RedirectMessage `protobuf:"bytes,2,opt,name=redirect" json:"redirect,omitempty"` + IsClosed bool `protobuf:"varint,3,opt,name=is_closed,json=isClosed" json:"is_closed,omitempty"` } func (m *PublishResponse) Reset() { *m = PublishResponse{} } @@ -338,6 +349,13 @@ func (m *PublishResponse) GetRedirect() *PublishResponse_RedirectMessage { return nil } +func (m *PublishResponse) GetIsClosed() bool { + if m != nil { + return m.IsClosed + } + return false +} + type PublishResponse_ConfigMessage struct { PartitionCount int32 `protobuf:"varint,1,opt,name=partition_count,json=partitionCount" json:"partition_count,omitempty"` } @@ -374,6 +392,38 @@ func (m *PublishResponse_RedirectMessage) GetNewBroker() string { return "" } +type DeleteTopicRequest struct { + Namespace string `protobuf:"bytes,1,opt,name=namespace" json:"namespace,omitempty"` + Topic string `protobuf:"bytes,2,opt,name=topic" json:"topic,omitempty"` +} + +func (m *DeleteTopicRequest) Reset() { *m = DeleteTopicRequest{} } +func (m *DeleteTopicRequest) String() string { return proto.CompactTextString(m) } +func (*DeleteTopicRequest) ProtoMessage() {} +func (*DeleteTopicRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } + +func (m *DeleteTopicRequest) GetNamespace() string { + if m != nil { + return m.Namespace + } + return "" +} + +func (m *DeleteTopicRequest) GetTopic() string { + if m != nil { + return m.Topic + } + return "" +} + +type DeleteTopicResponse struct { +} + +func (m *DeleteTopicResponse) Reset() { *m = DeleteTopicResponse{} } +func (m *DeleteTopicResponse) String() string { return proto.CompactTextString(m) } +func (*DeleteTopicResponse) ProtoMessage() {} +func (*DeleteTopicResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } + type ConfigureTopicRequest struct { Namespace string `protobuf:"bytes,1,opt,name=namespace" json:"namespace,omitempty"` Topic string `protobuf:"bytes,2,opt,name=topic" json:"topic,omitempty"` @@ -383,7 +433,7 @@ type ConfigureTopicRequest struct { func (m *ConfigureTopicRequest) Reset() { *m = ConfigureTopicRequest{} } func (m *ConfigureTopicRequest) String() string { return proto.CompactTextString(m) } func (*ConfigureTopicRequest) ProtoMessage() {} -func (*ConfigureTopicRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } +func (*ConfigureTopicRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } func (m *ConfigureTopicRequest) GetNamespace() string { if m != nil { @@ -412,7 +462,7 @@ type ConfigureTopicResponse struct { func (m *ConfigureTopicResponse) Reset() { *m = ConfigureTopicResponse{} } func (m *ConfigureTopicResponse) String() string { return proto.CompactTextString(m) } func (*ConfigureTopicResponse) ProtoMessage() {} -func (*ConfigureTopicResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } +func (*ConfigureTopicResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } type GetTopicConfigurationRequest struct { Namespace string `protobuf:"bytes,1,opt,name=namespace" json:"namespace,omitempty"` @@ -422,7 +472,7 @@ type GetTopicConfigurationRequest struct { func (m *GetTopicConfigurationRequest) Reset() { *m = GetTopicConfigurationRequest{} } func (m *GetTopicConfigurationRequest) String() string { return proto.CompactTextString(m) } func (*GetTopicConfigurationRequest) ProtoMessage() {} -func (*GetTopicConfigurationRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } +func (*GetTopicConfigurationRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} } func (m *GetTopicConfigurationRequest) GetNamespace() string { if m != nil { @@ -445,7 +495,7 @@ type GetTopicConfigurationResponse struct { func (m *GetTopicConfigurationResponse) Reset() { *m = GetTopicConfigurationResponse{} } func (m *GetTopicConfigurationResponse) String() string { return proto.CompactTextString(m) } func (*GetTopicConfigurationResponse) ProtoMessage() {} -func (*GetTopicConfigurationResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } +func (*GetTopicConfigurationResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } func (m *GetTopicConfigurationResponse) GetConfiguration() *TopicConfiguration { if m != nil { @@ -463,7 +513,7 @@ type FindBrokerRequest struct { func (m *FindBrokerRequest) Reset() { *m = FindBrokerRequest{} } func (m *FindBrokerRequest) String() string { return proto.CompactTextString(m) } func (*FindBrokerRequest) ProtoMessage() {} -func (*FindBrokerRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} } +func (*FindBrokerRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} } func (m *FindBrokerRequest) GetNamespace() string { if m != nil { @@ -493,7 +543,7 @@ type FindBrokerResponse struct { func (m *FindBrokerResponse) Reset() { *m = FindBrokerResponse{} } func (m *FindBrokerResponse) String() string { return proto.CompactTextString(m) } func (*FindBrokerResponse) ProtoMessage() {} -func (*FindBrokerResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } +func (*FindBrokerResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} } func (m *FindBrokerResponse) GetBroker() string { if m != nil { @@ -513,7 +563,7 @@ type TopicConfiguration struct { func (m *TopicConfiguration) Reset() { *m = TopicConfiguration{} } func (m *TopicConfiguration) String() string { return proto.CompactTextString(m) } func (*TopicConfiguration) ProtoMessage() {} -func (*TopicConfiguration) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} } +func (*TopicConfiguration) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} } func (m *TopicConfiguration) GetPartitionCount() int32 { if m != nil { @@ -561,6 +611,8 @@ func init() { proto.RegisterType((*PublishResponse)(nil), "messaging_pb.PublishResponse") proto.RegisterType((*PublishResponse_ConfigMessage)(nil), "messaging_pb.PublishResponse.ConfigMessage") proto.RegisterType((*PublishResponse_RedirectMessage)(nil), "messaging_pb.PublishResponse.RedirectMessage") + proto.RegisterType((*DeleteTopicRequest)(nil), "messaging_pb.DeleteTopicRequest") + proto.RegisterType((*DeleteTopicResponse)(nil), "messaging_pb.DeleteTopicResponse") proto.RegisterType((*ConfigureTopicRequest)(nil), "messaging_pb.ConfigureTopicRequest") proto.RegisterType((*ConfigureTopicResponse)(nil), "messaging_pb.ConfigureTopicResponse") proto.RegisterType((*GetTopicConfigurationRequest)(nil), "messaging_pb.GetTopicConfigurationRequest") @@ -585,6 +637,7 @@ const _ = grpc.SupportPackageIsVersion4 type SeaweedMessagingClient interface { Subscribe(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error) Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error) + DeleteTopic(ctx context.Context, in *DeleteTopicRequest, opts ...grpc.CallOption) (*DeleteTopicResponse, error) ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error) GetTopicConfiguration(ctx context.Context, in *GetTopicConfigurationRequest, opts ...grpc.CallOption) (*GetTopicConfigurationResponse, error) FindBroker(ctx context.Context, in *FindBrokerRequest, opts ...grpc.CallOption) (*FindBrokerResponse, error) @@ -660,6 +713,15 @@ func (x *seaweedMessagingPublishClient) Recv() (*PublishResponse, error) { return m, nil } +func (c *seaweedMessagingClient) DeleteTopic(ctx context.Context, in *DeleteTopicRequest, opts ...grpc.CallOption) (*DeleteTopicResponse, error) { + out := new(DeleteTopicResponse) + err := grpc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/DeleteTopic", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *seaweedMessagingClient) ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error) { out := new(ConfigureTopicResponse) err := grpc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/ConfigureTopic", in, out, c.cc, opts...) @@ -692,6 +754,7 @@ func (c *seaweedMessagingClient) FindBroker(ctx context.Context, in *FindBrokerR type SeaweedMessagingServer interface { Subscribe(SeaweedMessaging_SubscribeServer) error Publish(SeaweedMessaging_PublishServer) error + DeleteTopic(context.Context, *DeleteTopicRequest) (*DeleteTopicResponse, error) ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error) GetTopicConfiguration(context.Context, *GetTopicConfigurationRequest) (*GetTopicConfigurationResponse, error) FindBroker(context.Context, *FindBrokerRequest) (*FindBrokerResponse, error) @@ -753,6 +816,24 @@ func (x *seaweedMessagingPublishServer) Recv() (*PublishRequest, error) { return m, nil } +func _SeaweedMessaging_DeleteTopic_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteTopicRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingServer).DeleteTopic(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/messaging_pb.SeaweedMessaging/DeleteTopic", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingServer).DeleteTopic(ctx, req.(*DeleteTopicRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _SeaweedMessaging_ConfigureTopic_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(ConfigureTopicRequest) if err := dec(in); err != nil { @@ -811,6 +892,10 @@ var _SeaweedMessaging_serviceDesc = grpc.ServiceDesc{ ServiceName: "messaging_pb.SeaweedMessaging", HandlerType: (*SeaweedMessagingServer)(nil), Methods: []grpc.MethodDesc{ + { + MethodName: "DeleteTopic", + Handler: _SeaweedMessaging_DeleteTopic_Handler, + }, { MethodName: "ConfigureTopic", Handler: _SeaweedMessaging_ConfigureTopic_Handler, @@ -844,65 +929,68 @@ var _SeaweedMessaging_serviceDesc = grpc.ServiceDesc{ func init() { proto.RegisterFile("messaging.proto", fileDescriptor0) } var fileDescriptor0 = []byte{ - // 952 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x56, 0xdd, 0x6e, 0xe3, 0x44, - 0x14, 0xae, 0x9d, 0x34, 0x3f, 0x27, 0x3f, 0xcd, 0x1e, 0xd1, 0x55, 0x30, 0x2d, 0x04, 0x2f, 0x82, - 0x40, 0x21, 0xaa, 0xc2, 0x4d, 0x59, 0xad, 0xb4, 0x6a, 0xa3, 0x2e, 0x1b, 0xd1, 0x76, 0xc3, 0x24, - 0x5c, 0x22, 0xcb, 0xb1, 0x67, 0xd3, 0x51, 0x9d, 0xb1, 0xf1, 0x38, 0x5b, 0xf5, 0x9a, 0x6b, 0xae, - 0x78, 0x15, 0x5e, 0x80, 0x67, 0xe0, 0x02, 0x89, 0xa7, 0x41, 0x1e, 0xff, 0xc4, 0x4e, 0xb2, 0xe9, - 0x52, 0xb4, 0x77, 0x9e, 0x33, 0xdf, 0xf9, 0xce, 0x77, 0xce, 0x9c, 0x33, 0x63, 0xd8, 0x9b, 0x53, - 0x21, 0xcc, 0x19, 0xe3, 0xb3, 0x9e, 0xe7, 0xbb, 0x81, 0x8b, 0xf5, 0xd4, 0x60, 0x78, 0x53, 0xfd, - 0xd7, 0x22, 0x3c, 0x1a, 0x2f, 0xa6, 0xc2, 0xf2, 0xd9, 0x94, 0xfa, 0x97, 0x72, 0x8b, 0xe2, 0x73, - 0x28, 0x32, 0xce, 0x82, 0xb6, 0xd2, 0x51, 0xba, 0xb5, 0xfe, 0x51, 0x2f, 0xeb, 0xd2, 0x5b, 0x83, - 0xf7, 0x86, 0x9c, 0x05, 0xf1, 0x37, 0x91, 0x8e, 0xf8, 0x0c, 0x0a, 0xa6, 0x75, 0xd3, 0x56, 0xa5, - 0xff, 0x57, 0xf7, 0xf9, 0x9f, 0x5a, 0x37, 0x89, 0x7b, 0xe8, 0xa6, 0xfd, 0xa9, 0x42, 0x2d, 0xc3, - 0x89, 0x07, 0x50, 0xe5, 0xe6, 0x9c, 0x0a, 0xcf, 0xb4, 0xa8, 0xd4, 0x54, 0x25, 0x4b, 0x03, 0x7e, - 0x00, 0xbb, 0x81, 0xeb, 0x31, 0x4b, 0x46, 0xab, 0x92, 0x68, 0x11, 0xfa, 0x78, 0xa6, 0x1f, 0xb0, - 0x80, 0xb9, 0xbc, 0x5d, 0xe8, 0x28, 0xdd, 0x5d, 0xb2, 0x34, 0xa0, 0x01, 0x0d, 0x11, 0x98, 0x7e, - 0x30, 0x72, 0x45, 0x84, 0x28, 0x76, 0x94, 0x6e, 0xb3, 0xff, 0xdd, 0x7f, 0xc8, 0xb4, 0x37, 0xce, - 0x12, 0x90, 0x3c, 0x1f, 0x76, 0xa0, 0x16, 0xb0, 0x39, 0x15, 0x81, 0x39, 0xf7, 0xae, 0x44, 0x7b, - 0xb7, 0xa3, 0x74, 0x0b, 0x24, 0x6b, 0xc2, 0x27, 0xd0, 0x10, 0x29, 0xbf, 0xc1, 0xec, 0x76, 0x49, - 0xca, 0xaf, 0x2f, 0x8d, 0x43, 0x5b, 0x3f, 0x81, 0x46, 0x2e, 0x0c, 0x02, 0x94, 0x2e, 0x4e, 0x27, - 0xe7, 0xe3, 0x49, 0x6b, 0x07, 0xeb, 0x50, 0x39, 0x3f, 0x25, 0x17, 0xc3, 0x70, 0xa5, 0x60, 0x03, - 0xaa, 0x93, 0xe1, 0xe5, 0xf9, 0x78, 0x72, 0x7a, 0x39, 0x6a, 0xa9, 0xda, 0x11, 0xc0, 0xb2, 0xac, - 0x78, 0x08, 0x10, 0x65, 0x46, 0xc3, 0x48, 0x8a, 0x54, 0x53, 0x8d, 0x2d, 0x43, 0x5b, 0xff, 0x4b, - 0x81, 0x72, 0x02, 0xfd, 0x1c, 0x1a, 0xf4, 0x0d, 0xe5, 0x81, 0x11, 0x8a, 0x35, 0xb8, 0x88, 0xd0, - 0x67, 0xea, 0xb1, 0x42, 0x6a, 0x72, 0x63, 0xc2, 0xe6, 0xf4, 0x4a, 0x60, 0x0b, 0x0a, 0x37, 0xf4, - 0x4e, 0x16, 0xbd, 0x4e, 0xc2, 0xcf, 0xf0, 0x20, 0xde, 0x98, 0xce, 0x82, 0xca, 0x72, 0xd7, 0x49, - 0xb4, 0xc0, 0x67, 0x50, 0xbe, 0xa6, 0xa6, 0x4d, 0x7d, 0xd1, 0x2e, 0x76, 0x0a, 0xdd, 0x5a, 0x5f, - 0xcf, 0x17, 0x39, 0x29, 0xe7, 0xcb, 0x08, 0x74, 0xce, 0x03, 0xff, 0x8e, 0x24, 0x2e, 0xda, 0x53, - 0xa8, 0x67, 0x37, 0x92, 0xa8, 0x51, 0x13, 0xe4, 0xa3, 0xaa, 0x99, 0xa8, 0x4f, 0xd5, 0x13, 0x45, - 0xff, 0x09, 0x1a, 0x67, 0xbe, 0x7b, 0xb3, 0x6c, 0xeb, 0x2f, 0xa1, 0x68, 0x9b, 0x81, 0x19, 0xb7, - 0xf5, 0xfe, 0x46, 0x1d, 0x44, 0x42, 0xf0, 0x43, 0xa8, 0x30, 0x61, 0x58, 0x8e, 0x2b, 0x22, 0xe2, - 0x0a, 0x29, 0x33, 0x31, 0x08, 0x97, 0xfa, 0x3f, 0x0a, 0x34, 0x47, 0x8b, 0xa9, 0xc3, 0xc4, 0x35, - 0xa1, 0xbf, 0x2c, 0xa8, 0x08, 0xdb, 0x3d, 0x3b, 0x2f, 0xdd, 0x3c, 0x71, 0x1e, 0xbb, 0x61, 0x58, - 0x12, 0x59, 0xea, 0xbd, 0xb2, 0x34, 0xe3, 0x3d, 0x0f, 0x86, 0xfe, 0x9b, 0x0a, 0x7b, 0xa9, 0x60, - 0xe1, 0xb9, 0x5c, 0x50, 0x1c, 0x40, 0xc9, 0x72, 0xf9, 0x6b, 0x36, 0xdb, 0x7c, 0x1f, 0xac, 0xc0, - 0x7b, 0x03, 0x89, 0x4d, 0x74, 0xc7, 0xae, 0x38, 0x84, 0x8a, 0x4f, 0x6d, 0xe6, 0x53, 0x2b, 0x88, - 0x13, 0xfd, 0x66, 0x3b, 0x0d, 0x89, 0xd1, 0x09, 0x51, 0xea, 0xae, 0x9d, 0x40, 0x23, 0x17, 0x03, - 0xbf, 0x80, 0xbd, 0x34, 0x03, 0xc3, 0x72, 0x17, 0x3c, 0x3a, 0x89, 0x5d, 0xd2, 0x4c, 0xcd, 0x83, - 0xd0, 0xaa, 0x1d, 0xc3, 0xde, 0x0a, 0x6d, 0x38, 0x19, 0x9c, 0xde, 0x1a, 0x53, 0xd9, 0x28, 0x69, - 0x0d, 0xe9, 0x6d, 0xd4, 0x39, 0xfa, 0xef, 0x0a, 0xec, 0x47, 0xc1, 0x16, 0x3e, 0x9d, 0x84, 0x05, - 0x4c, 0xce, 0xfc, 0x21, 0xb5, 0x7f, 0x01, 0x0d, 0x2b, 0x26, 0x33, 0xd3, 0xfa, 0xd7, 0xfa, 0x9d, - 0x7c, 0x25, 0x64, 0x98, 0x41, 0x16, 0x47, 0xf2, 0x6e, 0x7a, 0x1b, 0x1e, 0xaf, 0x8a, 0x8a, 0xaa, - 0xa6, 0x13, 0x38, 0xf8, 0x9e, 0x06, 0x1b, 0x18, 0x1e, 0xae, 0x5a, 0x9f, 0xc1, 0xe1, 0x5b, 0x38, - 0xe3, 0x06, 0x59, 0x4b, 0x4b, 0x79, 0x58, 0x5a, 0x16, 0x3c, 0x7a, 0xc1, 0xb8, 0x1d, 0x95, 0xfe, - 0xff, 0xd4, 0x59, 0x83, 0x8a, 0x67, 0xfa, 0xd9, 0x16, 0x4f, 0xd7, 0xfa, 0xd7, 0x80, 0xd9, 0x20, - 0x71, 0x0a, 0x8f, 0xa1, 0x94, 0x6b, 0x81, 0x78, 0xa5, 0xff, 0xa1, 0x02, 0xae, 0x0b, 0x7f, 0xe7, - 0x8e, 0xc3, 0x8f, 0x01, 0x2c, 0xd7, 0x71, 0xa8, 0x25, 0xb5, 0x44, 0x22, 0x33, 0x96, 0xf0, 0x9d, - 0xf0, 0xa9, 0xe7, 0x30, 0x6b, 0xd9, 0x0f, 0x55, 0x92, 0x35, 0xe1, 0xa7, 0x50, 0x67, 0xc2, 0x08, - 0x7c, 0x93, 0x0b, 0x46, 0x79, 0x20, 0x5f, 0xaa, 0x0a, 0xa9, 0x31, 0x31, 0x49, 0x4c, 0xf8, 0x0a, - 0x6a, 0x51, 0x58, 0x97, 0x33, 0x3e, 0x93, 0x8f, 0x4d, 0x73, 0x75, 0xbc, 0xd6, 0x93, 0xe8, 0x8d, - 0x12, 0xa9, 0x8c, 0xcf, 0x48, 0x96, 0x41, 0x7f, 0x0e, 0xf5, 0xec, 0x26, 0x22, 0x34, 0xaf, 0x5c, - 0x7e, 0xb5, 0x70, 0x9c, 0x1f, 0xe8, 0xdd, 0x4b, 0x53, 0x5c, 0xb7, 0x76, 0xb0, 0x06, 0xe5, 0x64, - 0xa1, 0x60, 0x13, 0x80, 0xb8, 0x0b, 0x6e, 0x13, 0x77, 0xca, 0x78, 0x4b, 0xed, 0xff, 0x5d, 0x80, - 0xd6, 0x98, 0x9a, 0xb7, 0x94, 0xda, 0x97, 0x89, 0x0a, 0x7c, 0x05, 0xd5, 0xf4, 0x45, 0xc5, 0x4f, - 0xee, 0x79, 0x6a, 0xb5, 0x8f, 0xf2, 0x80, 0xdc, 0x4d, 0xae, 0xef, 0x74, 0x95, 0x63, 0x05, 0x2f, - 0xa0, 0x1c, 0xdf, 0x1a, 0x78, 0xb0, 0xed, 0xce, 0xd5, 0x0e, 0xb7, 0x5e, 0x35, 0x31, 0xdb, 0xcf, - 0xd0, 0xcc, 0x0f, 0x15, 0x3e, 0xc9, 0xbb, 0x6d, 0xbc, 0x07, 0xb4, 0xcf, 0xb6, 0x83, 0x92, 0x10, - 0xe8, 0xc3, 0xfe, 0xc6, 0x29, 0xc2, 0x95, 0xdf, 0xa3, 0x6d, 0xe3, 0xab, 0x1d, 0xbd, 0x13, 0x36, - 0x8d, 0xf9, 0x23, 0xc0, 0xb2, 0xd7, 0x57, 0x4b, 0xbe, 0x36, 0x6a, 0x5a, 0xe7, 0xed, 0x80, 0x84, - 0xf2, 0x4c, 0x87, 0x96, 0x88, 0x0e, 0xf6, 0xb5, 0xe8, 0x59, 0x4e, 0xd8, 0x7f, 0x67, 0xcd, 0xf4, - 0x8c, 0x47, 0xe1, 0x2f, 0xe6, 0xb4, 0x24, 0xff, 0x34, 0xbf, 0xfd, 0x37, 0x00, 0x00, 0xff, 0xff, - 0xad, 0x6b, 0x26, 0x8c, 0x7c, 0x0a, 0x00, 0x00, + // 1002 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x56, 0xdd, 0x6e, 0xe3, 0x54, + 0x10, 0xae, 0xdd, 0xfc, 0x8e, 0x93, 0x34, 0x3b, 0xd0, 0x55, 0xf0, 0xb6, 0x90, 0xf5, 0x22, 0x08, + 0x14, 0xa2, 0x2a, 0xdc, 0x94, 0x6a, 0xa5, 0x55, 0x1b, 0xba, 0x34, 0xa2, 0xed, 0x86, 0x93, 0xdc, + 0x22, 0xcb, 0xb1, 0xcf, 0xa6, 0x47, 0x75, 0x8e, 0x8d, 0x8f, 0xb3, 0x55, 0x9f, 0x83, 0x7b, 0x1e, + 0x00, 0x89, 0x3b, 0x5e, 0x80, 0xd7, 0xe0, 0x21, 0x78, 0x06, 0xe4, 0xdf, 0xd8, 0x49, 0x36, 0x5d, + 0xb6, 0xda, 0xbb, 0x9c, 0xc9, 0x37, 0x33, 0xdf, 0x99, 0xf9, 0x66, 0x8e, 0x61, 0x67, 0x46, 0x85, + 0x30, 0xa6, 0x8c, 0x4f, 0xbb, 0xae, 0xe7, 0xf8, 0x0e, 0xd6, 0x52, 0x83, 0xee, 0x4e, 0xb4, 0xdf, + 0x0b, 0xf0, 0x68, 0x34, 0x9f, 0x08, 0xd3, 0x63, 0x13, 0xea, 0x5d, 0x86, 0x7f, 0x51, 0x7c, 0x01, + 0x05, 0xc6, 0x99, 0xdf, 0x92, 0xda, 0x52, 0x47, 0xe9, 0x1d, 0x74, 0xb3, 0x2e, 0xdd, 0x15, 0x78, + 0x77, 0xc0, 0x99, 0x1f, 0xff, 0x26, 0xa1, 0x23, 0x3e, 0x87, 0x6d, 0xc3, 0xbc, 0x69, 0xc9, 0xa1, + 0xff, 0xd7, 0xf7, 0xf9, 0x9f, 0x98, 0x37, 0x89, 0x7b, 0xe0, 0x86, 0x9f, 0x40, 0x85, 0x09, 0xdd, + 0xb4, 0x1d, 0x41, 0x5b, 0xdb, 0x6d, 0xa9, 0x53, 0x21, 0x65, 0x26, 0xfa, 0xc1, 0x51, 0xfd, 0x5b, + 0x06, 0x25, 0x93, 0x0e, 0xf7, 0xa0, 0xca, 0x8d, 0x19, 0x15, 0xae, 0x61, 0xd2, 0x90, 0x6e, 0x95, + 0x2c, 0x0c, 0xf8, 0x31, 0x14, 0x7d, 0xc7, 0x65, 0x66, 0x48, 0xa4, 0x4a, 0xa2, 0x43, 0xe0, 0xe3, + 0x1a, 0x9e, 0xcf, 0x7c, 0xe6, 0xf0, 0x30, 0x7e, 0x91, 0x2c, 0x0c, 0xa8, 0x43, 0x5d, 0xf8, 0x86, + 0xe7, 0x0f, 0x1d, 0x11, 0x21, 0x0a, 0x6d, 0xa9, 0xd3, 0xe8, 0x7d, 0xff, 0x3f, 0x8a, 0xd0, 0x1d, + 0x65, 0x03, 0x90, 0x7c, 0x3c, 0x6c, 0x83, 0xe2, 0xb3, 0x19, 0x15, 0xbe, 0x31, 0x73, 0xaf, 0x44, + 0xab, 0xd8, 0x96, 0x3a, 0xdb, 0x24, 0x6b, 0xc2, 0x67, 0x50, 0x17, 0x69, 0x7c, 0x9d, 0x59, 0xad, + 0x52, 0x48, 0xbf, 0xb6, 0x30, 0x0e, 0x2c, 0xed, 0x08, 0xea, 0xb9, 0x34, 0x08, 0x50, 0xba, 0x38, + 0x19, 0x9f, 0x8d, 0xc6, 0xcd, 0x2d, 0xac, 0x41, 0xe5, 0xec, 0x84, 0x5c, 0x0c, 0x82, 0x93, 0x84, + 0x75, 0xa8, 0x8e, 0x07, 0x97, 0x67, 0xa3, 0xf1, 0xc9, 0xe5, 0xb0, 0x29, 0xab, 0x07, 0x00, 0x8b, + 0x8a, 0xe3, 0x3e, 0x40, 0x74, 0x33, 0x1a, 0x64, 0x92, 0x42, 0x36, 0xd5, 0xd8, 0x32, 0xb0, 0xb4, + 0x7f, 0x25, 0x28, 0x27, 0xd0, 0x2f, 0xa0, 0x4e, 0xdf, 0x50, 0xee, 0xeb, 0x01, 0x59, 0x9d, 0x8b, + 0x08, 0x7d, 0x2a, 0x1f, 0x4a, 0x44, 0x09, 0xff, 0x18, 0xb3, 0x19, 0xbd, 0x12, 0xd8, 0x84, 0xed, + 0x1b, 0x7a, 0x17, 0x16, 0xbd, 0x46, 0x82, 0x9f, 0x41, 0x23, 0xde, 0x18, 0xf6, 0x3c, 0x6a, 0x67, + 0x8d, 0x44, 0x07, 0x7c, 0x0e, 0xe5, 0x6b, 0x6a, 0x58, 0xd4, 0x13, 0xad, 0x42, 0x7b, 0xbb, 0xa3, + 0xf4, 0xb4, 0x7c, 0x91, 0x93, 0x72, 0x9e, 0x47, 0xa0, 0x33, 0xee, 0x7b, 0x77, 0x24, 0x71, 0xc9, + 0xa9, 0xa4, 0x98, 0x57, 0xc9, 0x31, 0xd4, 0xb2, 0x3e, 0x09, 0xa1, 0x48, 0x1f, 0x79, 0x42, 0x72, + 0x86, 0xd0, 0xb1, 0x7c, 0x24, 0x69, 0xc7, 0x50, 0x3f, 0xf5, 0x9c, 0x9b, 0xc5, 0x30, 0x7c, 0x05, + 0x05, 0xcb, 0xf0, 0x8d, 0x78, 0x18, 0x76, 0xd7, 0x52, 0x24, 0x21, 0x44, 0xfb, 0x47, 0x82, 0xc6, + 0x70, 0x3e, 0xb1, 0x99, 0xb8, 0x26, 0xf4, 0xd7, 0x39, 0x15, 0xc1, 0x24, 0x64, 0x47, 0xa9, 0x93, + 0xf7, 0xce, 0x63, 0xd7, 0xcc, 0x51, 0x92, 0x5b, 0xbe, 0x37, 0xb7, 0xaa, 0x7f, 0xe0, 0xc1, 0xd0, + 0xfe, 0x90, 0x61, 0x27, 0x25, 0x2c, 0x5c, 0x87, 0x0b, 0x8a, 0x7d, 0x28, 0x99, 0x0e, 0x7f, 0xcd, + 0xa6, 0xeb, 0x57, 0xc5, 0x12, 0xbc, 0xdb, 0x0f, 0xb1, 0x09, 0xef, 0xd8, 0x15, 0x07, 0x50, 0xf1, + 0xa8, 0xc5, 0x3c, 0x6a, 0xfa, 0xf1, 0x45, 0xbf, 0xdd, 0x1c, 0x86, 0xc4, 0xe8, 0x24, 0x50, 0xea, + 0x8e, 0x4f, 0xa0, 0x9a, 0x68, 0xc2, 0x8a, 0x57, 0x47, 0x25, 0x16, 0x85, 0xa5, 0x1e, 0x41, 0x3d, + 0x47, 0x00, 0xbf, 0x84, 0x9d, 0xf4, 0x7a, 0xba, 0xe9, 0xcc, 0x79, 0xd4, 0xa6, 0x22, 0x69, 0xa4, + 0xe6, 0x7e, 0x60, 0x55, 0x0f, 0x61, 0x67, 0x29, 0x67, 0x30, 0x36, 0x9c, 0xde, 0xea, 0x93, 0x50, + 0x2a, 0x69, 0x81, 0xe9, 0x6d, 0xa4, 0x1d, 0xed, 0x1c, 0xf0, 0x07, 0x6a, 0x53, 0x9f, 0x8e, 0x83, + 0xca, 0x26, 0x62, 0x78, 0x8f, 0xa6, 0x68, 0xbb, 0xf0, 0x51, 0x2e, 0x52, 0x54, 0x03, 0xed, 0x37, + 0x09, 0x76, 0xa3, 0xdb, 0xcc, 0xbd, 0x07, 0x27, 0xc1, 0x97, 0x50, 0x37, 0xe3, 0x60, 0x46, 0xda, + 0x7d, 0xa5, 0xd7, 0xce, 0xf7, 0x21, 0x4c, 0xd3, 0xcf, 0xe2, 0x48, 0xde, 0x4d, 0x6b, 0xc1, 0xe3, + 0x65, 0x52, 0x31, 0x5f, 0x02, 0x7b, 0x3f, 0x52, 0x7f, 0x4d, 0x84, 0x07, 0x94, 0x66, 0x0a, 0xfb, + 0x6f, 0x89, 0x19, 0xcb, 0x73, 0xe5, 0x5a, 0xd2, 0xfb, 0x5d, 0xcb, 0x84, 0x47, 0x2f, 0x19, 0xb7, + 0xa2, 0xde, 0x3e, 0xa4, 0xce, 0x2a, 0x54, 0x5c, 0xc3, 0xcb, 0x0e, 0x58, 0x7a, 0xd6, 0xbe, 0x01, + 0xcc, 0x26, 0x89, 0xaf, 0xf0, 0x18, 0x4a, 0x39, 0x8d, 0xc5, 0x27, 0xed, 0x2f, 0x19, 0x70, 0x95, + 0xf8, 0x3b, 0x4b, 0x1a, 0x3f, 0x05, 0x30, 0x1d, 0xdb, 0xa6, 0x66, 0xc8, 0x25, 0x22, 0x99, 0xb1, + 0x04, 0xaf, 0x94, 0x47, 0x5d, 0x9b, 0x99, 0x0b, 0x3d, 0x54, 0x49, 0xd6, 0x84, 0x4f, 0xa1, 0xc6, + 0x84, 0xee, 0x7b, 0x06, 0x17, 0x8c, 0x72, 0x3f, 0x7c, 0x27, 0x2b, 0x44, 0x61, 0x62, 0x9c, 0x98, + 0xf0, 0x15, 0x28, 0x51, 0x5a, 0x87, 0x33, 0x3e, 0x0d, 0xb7, 0x74, 0x63, 0x79, 0xb8, 0x57, 0x2f, + 0xd1, 0x1d, 0x26, 0x54, 0x19, 0x9f, 0x92, 0x6c, 0x04, 0xed, 0x05, 0xd4, 0xb2, 0x7f, 0x22, 0x42, + 0xe3, 0xca, 0xe1, 0x57, 0x73, 0xdb, 0xfe, 0x89, 0xde, 0x9d, 0x1b, 0xe2, 0xba, 0xb9, 0x85, 0x0a, + 0x94, 0x93, 0x83, 0x84, 0x0d, 0x00, 0xe2, 0xcc, 0xb9, 0x45, 0x9c, 0x09, 0xe3, 0x4d, 0xb9, 0xf7, + 0x67, 0x01, 0x9a, 0x23, 0x6a, 0xdc, 0x52, 0x6a, 0x5d, 0x26, 0x2c, 0xf0, 0x15, 0x54, 0xd3, 0xf7, + 0x1c, 0x3f, 0xbb, 0xe7, 0xa1, 0x57, 0x9f, 0xe4, 0x01, 0xb9, 0xc7, 0x42, 0xdb, 0xea, 0x48, 0x87, + 0x12, 0x5e, 0x40, 0x39, 0xde, 0x59, 0xb8, 0xb7, 0x69, 0xe3, 0xab, 0xfb, 0x1b, 0x17, 0x5d, 0x1c, + 0x6d, 0x0c, 0x4a, 0x66, 0x03, 0xe0, 0x92, 0x7a, 0x57, 0xd7, 0x8c, 0xfa, 0x74, 0x03, 0x22, 0x89, + 0x8c, 0xbf, 0x40, 0x23, 0x3f, 0xaa, 0xf8, 0x2c, 0xef, 0xb6, 0x76, 0xbb, 0xa8, 0x9f, 0x6f, 0x06, + 0xa5, 0xe1, 0x3d, 0xd8, 0x5d, 0x3b, 0x9b, 0xb8, 0xf4, 0x35, 0xb8, 0x69, 0x29, 0xa8, 0x07, 0xef, + 0x84, 0x4d, 0x73, 0xfe, 0x0c, 0xb0, 0x98, 0xa0, 0xe5, 0x46, 0xae, 0x0c, 0xb0, 0xda, 0x7e, 0x3b, + 0x20, 0x09, 0x79, 0xaa, 0x41, 0x53, 0x44, 0x72, 0x79, 0x2d, 0xba, 0xa6, 0x1d, 0xa8, 0xfa, 0xb4, + 0x91, 0x2a, 0x67, 0x18, 0x7c, 0x51, 0x4f, 0x4a, 0xe1, 0x87, 0xf5, 0x77, 0xff, 0x05, 0x00, 0x00, + 0xff, 0xff, 0x7f, 0x62, 0xba, 0x48, 0x6b, 0x0b, 0x00, 0x00, } diff --git a/weed/server/filer_grpc_server.go b/weed/server/filer_grpc_server.go index 266030f5d..901f798f0 100644 --- a/weed/server/filer_grpc_server.go +++ b/weed/server/filer_grpc_server.go @@ -390,8 +390,12 @@ func (fs *FilerServer) KeepConnected(stream filer_pb.SeaweedFiler_KeepConnectedS } clientName := fmt.Sprintf("%s:%d", req.Name, req.GrpcPort) + m := make(map[string]bool) + for _, tp := range req.Resources { + m[tp] = true + } fs.brokersLock.Lock() - fs.brokers[clientName] = true + fs.brokers[clientName] = m glog.V(0).Infof("+ broker %v", clientName) fs.brokersLock.Unlock() @@ -417,3 +421,35 @@ func (fs *FilerServer) KeepConnected(stream filer_pb.SeaweedFiler_KeepConnectedS } } + +func (fs *FilerServer) LocateBroker(ctx context.Context, req *filer_pb.LocateBrokerRequest) (resp *filer_pb.LocateBrokerResponse, err error) { + + resp = &filer_pb.LocateBrokerResponse{} + + fs.brokersLock.Lock() + defer fs.brokersLock.Unlock() + + var localBrokers []*filer_pb.LocateBrokerResponse_Resource + + for b, m := range fs.brokers { + if _, found := m[req.Resource]; found { + resp.Found = true + resp.Resources = []*filer_pb.LocateBrokerResponse_Resource{ + { + GrpcAddresses: b, + ResourceCount: int32(len(m)), + }, + } + return + } + localBrokers = append(localBrokers, &filer_pb.LocateBrokerResponse_Resource{ + GrpcAddresses: b, + ResourceCount: int32(len(m)), + }) + } + + resp.Resources = localBrokers + + return resp, nil + +} diff --git a/weed/server/filer_grpc_server_listen.go b/weed/server/filer_grpc_server_listen.go index e45068f39..848a1fc3a 100644 --- a/weed/server/filer_grpc_server_listen.go +++ b/weed/server/filer_grpc_server_listen.go @@ -82,7 +82,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest, lastReadTime = time.Unix(0, processedTsNs) } - _, err := fs.filer.MetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool { + err := fs.filer.MetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool { fs.listenersLock.Lock() fs.listenersCond.Wait() fs.listenersLock.Unlock() diff --git a/weed/server/filer_server.go b/weed/server/filer_server.go index 0a54ea97d..10b607dfe 100644 --- a/weed/server/filer_server.go +++ b/weed/server/filer_server.go @@ -64,7 +64,7 @@ type FilerServer struct { listenersLock sync.Mutex listenersCond *sync.Cond - brokers map[string]bool + brokers map[string]map[string]bool brokersLock sync.Mutex } @@ -73,7 +73,7 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption) fs = &FilerServer{ option: option, grpcDialOption: security.LoadClientTLS(util.GetViper(), "grpc.filer"), - brokers: make(map[string]bool), + brokers: make(map[string]map[string]bool), } fs.listenersCond = sync.NewCond(&fs.listenersLock) diff --git a/weed/util/log_buffer/log_read.go b/weed/util/log_buffer/log_read.go index 6339d9d77..a2345acfb 100644 --- a/weed/util/log_buffer/log_read.go +++ b/weed/util/log_buffer/log_read.go @@ -12,8 +12,9 @@ import ( ) func (logBuffer *LogBuffer) LoopProcessLogData( - startTreadTime time.Time, waitForDataFn func() bool, - eachLogDataFn func(logEntry *filer_pb.LogEntry) error) (processed int64, err error) { + startTreadTime time.Time, + waitForDataFn func() bool, + eachLogDataFn func(logEntry *filer_pb.LogEntry) error) (err error) { // loop through all messages var bytesBuf *bytes.Buffer lastReadTime := startTreadTime @@ -66,7 +67,6 @@ func (logBuffer *LogBuffer) LoopProcessLogData( pos += 4 + int(size) batchSize++ - processed++ } // fmt.Printf("sent message ts[%d,%d] size %d\n", startReadTime.UnixNano(), lastReadTime.UnixNano(), batchSize)