1
0
Fork 0
mirror of https://github.com/chrislusf/seaweedfs synced 2024-07-03 23:56:41 +02:00

able to read chan and write chan

This commit is contained in:
Chris Lu 2020-05-08 02:47:22 -07:00
parent a8bc8eb351
commit dfccc3c263
23 changed files with 734 additions and 260 deletions

2
go.mod
View file

@ -10,6 +10,8 @@ require (
github.com/OneOfOne/xxhash v1.2.2 github.com/OneOfOne/xxhash v1.2.2
github.com/Shopify/sarama v1.23.1 github.com/Shopify/sarama v1.23.1
github.com/aws/aws-sdk-go v1.23.13 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/chrislusf/raft v0.0.0-20190225081310-10d6e2182d92
github.com/coreos/bbolt v1.3.3 // indirect github.com/coreos/bbolt v1.3.3 // indirect
github.com/coreos/etcd v3.3.15+incompatible // indirect github.com/coreos/etcd v3.3.15+incompatible // indirect

2
go.sum
View file

@ -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/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 h1:DDGfHa7BWjL4YnC6+E63dPcxHo2sUxDIu8g3QgEJdRY=
github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4= 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 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.0/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
github.com/census-instrumentation/opencensus-proto v0.2.1 h1:glEXhBS5PSLLv4IXzLA5yPRVX4bilULVyxxbrfOtDAk= github.com/census-instrumentation/opencensus-proto v0.2.1 h1:glEXhBS5PSLLv4IXzLA5yPRVX4bilULVyxxbrfOtDAk=

View file

@ -50,6 +50,10 @@ service SeaweedFiler {
rpc KeepConnected (stream KeepConnectedRequest) returns (stream KeepConnectedResponse) { rpc KeepConnected (stream KeepConnectedRequest) returns (stream KeepConnectedResponse) {
} }
rpc LocateBroker (LocateBrokerRequest) returns (LocateBrokerResponse) {
}
} }
////////////////////////////////////////////////// //////////////////////////////////////////////////
@ -267,6 +271,21 @@ message LogEntry {
message KeepConnectedRequest { message KeepConnectedRequest {
string name = 1; string name = 1;
uint32 grpc_port = 2; uint32 grpc_port = 2;
repeated string resources = 3;
} }
message KeepConnectedResponse { 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;
}

View file

@ -3,6 +3,7 @@ package broker
import ( import (
"context" "context"
"fmt" "fmt"
"io"
"github.com/chrislusf/seaweedfs/weed/glog" "github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/operation" "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 { for _, filer := range broker.option.Filers {
if err = pb.WithFilerClient(filer, broker.grpcDialOption, fn); err != nil { 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) glog.V(0).Infof("fail to connect to %s: %v", filer, err)
} else { } else {
break break

View file

@ -10,6 +10,10 @@ func (broker *MessageBroker) ConfigureTopic(c context.Context, request *messagin
panic("implement me") 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) { func (broker *MessageBroker) GetTopicConfiguration(c context.Context, request *messaging_pb.GetTopicConfigurationRequest) (*messaging_pb.GetTopicConfigurationResponse, error) {
panic("implement me") panic("implement me")
} }

View file

@ -2,6 +2,7 @@ package broker
import ( import (
"context" "context"
"fmt"
"time" "time"
"github.com/chrislusf/seaweedfs/weed/glog" "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) { 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) checkFilers() {
func (broker *MessageBroker) checkPeers() {
// contact a filer about masters // contact a filer about masters
var masters []string var masters []string

View file

@ -47,24 +47,11 @@ func (broker *MessageBroker) Publish(stream messaging_pb.SeaweedMessaging_Publis
tl := broker.topicLocks.RequestLock(tp, topicConfig, true) tl := broker.topicLocks.RequestLock(tp, topicConfig, true)
defer broker.topicLocks.ReleaseLock(tp, 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 // process each message
for { for {
// println("recv")
in, err := stream.Recv() in, err := stream.Recv()
// glog.V(0).Infof("recieved %v err: %v", in, err)
if err == io.EOF { if err == io.EOF {
return nil return nil
} }
@ -86,5 +73,18 @@ func (broker *MessageBroker) Publish(stream messaging_pb.SeaweedMessaging_Publis
tl.logBuffer.AddToBuffer(in.Data.Key, data) 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
} }

View file

@ -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) glog.Errorf("sending %d bytes to %s: %s", len(m.Value), subscriberId, err)
return err return err
} }
if m.IsClose {
// println("processed EOF")
return io.EOF
}
processedTsNs = logEntry.TsNs processedTsNs = logEntry.TsNs
messageCount++
return nil return nil
} }
if err := broker.readPersistedLogBuffer(&tp, lastReadTime, eachLogEntryFn); err != nil { if err := broker.readPersistedLogBuffer(&tp, lastReadTime, eachLogEntryFn); err != nil {
// println("stopping from persisted logs")
return err return err
} }
@ -95,7 +101,7 @@ func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_Subs
lastReadTime = time.Unix(0, processedTsNs) lastReadTime = time.Unix(0, processedTsNs)
} }
messageCount, err = lock.logBuffer.LoopProcessLogData(lastReadTime, func() bool { err = lock.logBuffer.LoopProcessLogData(lastReadTime, func() bool {
lock.Mutex.Lock() lock.Mutex.Lock()
lock.cond.Wait() lock.cond.Wait()
lock.Mutex.Unlock() lock.Mutex.Unlock()
@ -124,7 +130,7 @@ func (broker *MessageBroker) readPersistedLogBuffer(tp *TopicPartition, startTim
return nil return nil
} }
} }
if !strings.HasSuffix(hourMinuteEntry.Name, partitionSuffix){ if !strings.HasSuffix(hourMinuteEntry.Name, partitionSuffix) {
return nil return nil
} }
// println("partition", tp.Partition, "processing", dayDir, "/", hourMinuteEntry.Name) // 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 { if err := filer2.ReadEachLogEntry(chunkedFileReader, sizeBuf, startTsNs, eachLogEntryFn); err != nil {
chunkedFileReader.Close() chunkedFileReader.Close()
if err == io.EOF { if err == io.EOF {
return nil return err
} }
return fmt.Errorf("reading %s/%s: %v", dayDir, hourMinuteEntry.Name, err) return fmt.Errorf("reading %s/%s: %v", dayDir, hourMinuteEntry.Name, err)
} }

View file

@ -36,7 +36,7 @@ func NewMessageBroker(option *MessageBrokerOption, grpcDialOption grpc.DialOptio
messageBroker.topicLocks = NewTopicLocks(messageBroker) messageBroker.topicLocks = NewTopicLocks(messageBroker)
messageBroker.checkPeers() messageBroker.checkFilers()
go messageBroker.keepConnectedToOneFiler() 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) glog.V(0).Infof("%s:%d failed to keep connected to %s: %v", broker.option.Ip, broker.option.Port, filer, err)
return 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) glog.V(0).Infof("conntected with filer: %v", filer)
for { for {
if err := stream.Send(&filer_pb.KeepConnectedRequest{ if err := stream.Send(&filer_pb.KeepConnectedRequest{
@ -68,12 +86,12 @@ func (broker *MessageBroker) keepConnectedToOneFiler() {
return err return err
} }
// println("received reply") // println("received reply")
time.Sleep(11*time.Second) time.Sleep(11 * time.Second)
// println("woke up") // println("woke up")
} }
return nil return nil
}) })
time.Sleep(3*time.Second) time.Sleep(3 * time.Second)
} }
} }

View file

@ -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()
}

View file

@ -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<total;i++{
tp := fmt.Sprintf("tp:%2d", i)
m := PickMember(servers, []byte(tp))
// println(tp, "=>", 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)))
}
}

View file

@ -16,6 +16,13 @@ type TopicPartition struct {
Topic string Topic string
Partition int32 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 { type TopicLock struct {
sync.Mutex sync.Mutex
cond *sync.Cond cond *sync.Cond
@ -101,3 +108,13 @@ func (tl *TopicLocks) ReleaseLock(partition TopicPartition, isPublisher bool) {
delete(tl.locks, partition) 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
}

View file

@ -5,12 +5,15 @@ import (
"log" "log"
"time" "time"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/messaging/broker" "github.com/chrislusf/seaweedfs/weed/messaging/broker"
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb" "github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
) )
type PubChannel struct { type PubChannel struct {
client messaging_pb.SeaweedMessaging_PublishClient client messaging_pb.SeaweedMessaging_PublishClient
grpcConnection *grpc.ClientConn
} }
func (mc *MessagingClient) NewPubChannel(chanName string) (*PubChannel, error) { func (mc *MessagingClient) NewPubChannel(chanName string) (*PubChannel, error) {
@ -28,7 +31,8 @@ func (mc *MessagingClient) NewPubChannel(chanName string) (*PubChannel, error) {
return nil, err return nil, err
} }
return &PubChannel{ return &PubChannel{
client: pc, client: pc,
grpcConnection: grpcConnection,
}, nil }, nil
} }
@ -40,7 +44,24 @@ func (pc *PubChannel) Publish(m []byte) error {
}) })
} }
func (pc *PubChannel) Close() 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 { type SubChannel struct {
@ -58,7 +79,7 @@ func (mc *MessagingClient) NewSubChannel(chanName string) (*SubChannel, error) {
if err != nil { if err != nil {
return nil, err 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 { if err != nil {
return nil, err 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) log.Printf("fail to receive from netchan %s: %v", chanName, subErr)
return return
} }
if resp.IsClose { if resp.Data.IsClose {
t.stream.Send(&messaging_pb.SubscriberMessage{
IsClose: true,
})
close(t.ch) close(t.ch)
return return
} }
if resp.Data != nil { t.ch <- resp.Data.Value
t.ch <- resp.Data.Value
}
} }
}() }()

View file

@ -4,9 +4,9 @@ import (
"context" "context"
"github.com/OneOfOne/xxhash" "github.com/OneOfOne/xxhash"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/messaging/broker" "github.com/chrislusf/seaweedfs/weed/messaging/broker"
"github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb" "github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
) )
@ -16,7 +16,7 @@ type Publisher struct {
messageCount uint64 messageCount uint64
publisherId string publisherId string
} }
/*
func (mc *MessagingClient) NewPublisher(publisherId, namespace, topic string) (*Publisher, error) { func (mc *MessagingClient) NewPublisher(publisherId, namespace, topic string) (*Publisher, error) {
// read topic configuration // read topic configuration
topicConfiguration := &messaging_pb.TopicConfiguration{ topicConfiguration := &messaging_pb.TopicConfiguration{
@ -24,7 +24,11 @@ func (mc *MessagingClient) NewPublisher(publisherId, namespace, topic string) (*
} }
publishClients := make([]messaging_pb.SeaweedMessaging_PublishClient, topicConfiguration.PartitionCount) publishClients := make([]messaging_pb.SeaweedMessaging_PublishClient, topicConfiguration.PartitionCount)
for i := 0; i < int(topicConfiguration.PartitionCount); i++ { 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 { if err != nil {
return nil, err return nil, err
} }
@ -35,6 +39,7 @@ func (mc *MessagingClient) NewPublisher(publisherId, namespace, topic string) (*
topicConfiguration: topicConfiguration, topicConfiguration: topicConfiguration,
}, nil }, nil
} }
*/
func setupPublisherClient(grpcConnection *grpc.ClientConn, tp broker.TopicPartition) (messaging_pb.SeaweedMessaging_PublishClient, error) { func setupPublisherClient(grpcConnection *grpc.ClientConn, tp broker.TopicPartition) (messaging_pb.SeaweedMessaging_PublishClient, error) {

View file

@ -5,6 +5,7 @@ import (
"io" "io"
"time" "time"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/pb/messaging_pb" "github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
) )
@ -13,6 +14,7 @@ type Subscriber struct {
subscriberId string subscriberId string
} }
/*
func (mc *MessagingClient) NewSubscriber(subscriberId, namespace, topic string, startTime time.Time) (*Subscriber, error) { func (mc *MessagingClient) NewSubscriber(subscriberId, namespace, topic string, startTime time.Time) (*Subscriber, error) {
// read topic configuration // read topic configuration
topicConfiguration := &messaging_pb.TopicConfiguration{ 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) { 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 { if err != nil {
return client, err return stream, err
} }
if newBroker != nil { if newBroker != nil {
@ -47,6 +49,7 @@ func (mc *MessagingClient) setupSubscriberClient(subscriberId, namespace, topic
return stream, nil 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) { 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()) stream, err = messaging_pb.NewSeaweedMessagingClient(grpcConnection).Subscribe(context.Background())
@ -70,13 +73,10 @@ func setupSubscriberClient(grpcConnection *grpc.ClientConn, subscriberId string,
} }
// process init response // process init response
initResponse, err := stream.Recv() _, err = stream.Recv()
if err != nil { if err != nil {
return return
} }
if initResponse.Redirect != nil {
// TODO follow redirection
}
return stream, nil return stream, nil
} }

View file

@ -50,6 +50,10 @@ service SeaweedFiler {
rpc KeepConnected (stream KeepConnectedRequest) returns (stream KeepConnectedResponse) { rpc KeepConnected (stream KeepConnectedRequest) returns (stream KeepConnectedResponse) {
} }
rpc LocateBroker (LocateBrokerRequest) returns (LocateBrokerResponse) {
}
} }
////////////////////////////////////////////////// //////////////////////////////////////////////////
@ -267,6 +271,21 @@ message LogEntry {
message KeepConnectedRequest { message KeepConnectedRequest {
string name = 1; string name = 1;
uint32 grpc_port = 2; uint32 grpc_port = 2;
repeated string resources = 3;
} }
message KeepConnectedResponse { 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;
}

View file

@ -46,6 +46,8 @@ It has these top-level messages:
LogEntry LogEntry
KeepConnectedRequest KeepConnectedRequest
KeepConnectedResponse KeepConnectedResponse
LocateBrokerRequest
LocateBrokerResponse
*/ */
package filer_pb package filer_pb
@ -1231,8 +1233,9 @@ func (m *LogEntry) GetData() []byte {
} }
type KeepConnectedRequest struct { type KeepConnectedRequest struct {
Name string `protobuf:"bytes,1,opt,name=name" json:"name,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"` 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{} } func (m *KeepConnectedRequest) Reset() { *m = KeepConnectedRequest{} }
@ -1254,6 +1257,13 @@ func (m *KeepConnectedRequest) GetGrpcPort() uint32 {
return 0 return 0
} }
func (m *KeepConnectedRequest) GetResources() []string {
if m != nil {
return m.Resources
}
return nil
}
type KeepConnectedResponse struct { type KeepConnectedResponse struct {
} }
@ -1262,6 +1272,74 @@ func (m *KeepConnectedResponse) String() string { return proto.Compac
func (*KeepConnectedResponse) ProtoMessage() {} func (*KeepConnectedResponse) ProtoMessage() {}
func (*KeepConnectedResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{36} } 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() { func init() {
proto.RegisterType((*LookupDirectoryEntryRequest)(nil), "filer_pb.LookupDirectoryEntryRequest") proto.RegisterType((*LookupDirectoryEntryRequest)(nil), "filer_pb.LookupDirectoryEntryRequest")
proto.RegisterType((*LookupDirectoryEntryResponse)(nil), "filer_pb.LookupDirectoryEntryResponse") proto.RegisterType((*LookupDirectoryEntryResponse)(nil), "filer_pb.LookupDirectoryEntryResponse")
@ -1300,6 +1378,9 @@ func init() {
proto.RegisterType((*LogEntry)(nil), "filer_pb.LogEntry") proto.RegisterType((*LogEntry)(nil), "filer_pb.LogEntry")
proto.RegisterType((*KeepConnectedRequest)(nil), "filer_pb.KeepConnectedRequest") proto.RegisterType((*KeepConnectedRequest)(nil), "filer_pb.KeepConnectedRequest")
proto.RegisterType((*KeepConnectedResponse)(nil), "filer_pb.KeepConnectedResponse") 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. // 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) GetFilerConfiguration(ctx context.Context, in *GetFilerConfigurationRequest, opts ...grpc.CallOption) (*GetFilerConfigurationResponse, error)
SubscribeMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeMetadataClient, error) SubscribeMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeMetadataClient, error)
KeepConnected(ctx context.Context, opts ...grpc.CallOption) (SeaweedFiler_KeepConnectedClient, 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 { type seaweedFilerClient struct {
@ -1531,6 +1613,15 @@ func (x *seaweedFilerKeepConnectedClient) Recv() (*KeepConnectedResponse, error)
return m, nil 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 // Server API for SeaweedFiler service
type SeaweedFilerServer interface { type SeaweedFilerServer interface {
@ -1548,6 +1639,7 @@ type SeaweedFilerServer interface {
GetFilerConfiguration(context.Context, *GetFilerConfigurationRequest) (*GetFilerConfigurationResponse, error) GetFilerConfiguration(context.Context, *GetFilerConfigurationRequest) (*GetFilerConfigurationResponse, error)
SubscribeMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeMetadataServer) error SubscribeMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeMetadataServer) error
KeepConnected(SeaweedFiler_KeepConnectedServer) error KeepConnected(SeaweedFiler_KeepConnectedServer) error
LocateBroker(context.Context, *LocateBrokerRequest) (*LocateBrokerResponse, error)
} }
func RegisterSeaweedFilerServer(s *grpc.Server, srv SeaweedFilerServer) { func RegisterSeaweedFilerServer(s *grpc.Server, srv SeaweedFilerServer) {
@ -1820,6 +1912,24 @@ func (x *seaweedFilerKeepConnectedServer) Recv() (*KeepConnectedRequest, error)
return m, nil 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{ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
ServiceName: "filer_pb.SeaweedFiler", ServiceName: "filer_pb.SeaweedFiler",
HandlerType: (*SeaweedFilerServer)(nil), HandlerType: (*SeaweedFilerServer)(nil),
@ -1868,6 +1978,10 @@ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
MethodName: "GetFilerConfiguration", MethodName: "GetFilerConfiguration",
Handler: _SeaweedFiler_GetFilerConfiguration_Handler, Handler: _SeaweedFiler_GetFilerConfiguration_Handler,
}, },
{
MethodName: "LocateBroker",
Handler: _SeaweedFiler_LocateBroker_Handler,
},
}, },
Streams: []grpc.StreamDesc{ Streams: []grpc.StreamDesc{
{ {
@ -1893,132 +2007,139 @@ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
func init() { proto.RegisterFile("filer.proto", fileDescriptor0) } func init() { proto.RegisterFile("filer.proto", fileDescriptor0) }
var fileDescriptor0 = []byte{ var fileDescriptor0 = []byte{
// 2020 bytes of a gzipped FileDescriptorProto // 2142 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x58, 0xcd, 0x6e, 0xdc, 0xc8, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x59, 0x5f, 0x6f, 0xdb, 0xc8,
0x11, 0x16, 0x67, 0x34, 0xa3, 0x61, 0xcd, 0x8c, 0x57, 0x6a, 0x49, 0xf6, 0x68, 0xf4, 0x63, 0x2d, 0x11, 0x37, 0x25, 0x4b, 0x16, 0x47, 0x52, 0xce, 0x5e, 0x3b, 0x89, 0xa2, 0xc4, 0x89, 0x8f, 0x69,
0x1d, 0x6f, 0x14, 0xd8, 0x50, 0x0c, 0x65, 0x03, 0xec, 0x66, 0x93, 0x83, 0x2d, 0xcb, 0x8e, 0x63, 0xee, 0x5c, 0x24, 0x70, 0x53, 0xf7, 0x0a, 0xdc, 0xf5, 0xda, 0x07, 0xc7, 0x71, 0xae, 0x69, 0x12,
0x5b, 0x2b, 0x50, 0xf6, 0x22, 0x41, 0x80, 0x30, 0x14, 0xd9, 0x1a, 0x75, 0xc4, 0x21, 0x99, 0xee, 0x9f, 0x41, 0x27, 0x87, 0x2b, 0x0a, 0x94, 0xa5, 0xc9, 0xb5, 0xbc, 0x35, 0x45, 0xb2, 0xbb, 0x4b,
0xa6, 0x7e, 0xf6, 0xb4, 0xcf, 0x11, 0x20, 0x6f, 0x91, 0x63, 0x90, 0x4b, 0x10, 0x20, 0x40, 0xce, 0xff, 0xb9, 0xa7, 0xfb, 0x1c, 0x05, 0xfa, 0xda, 0x4f, 0xd0, 0xc7, 0xa2, 0x2f, 0x45, 0x81, 0x02,
0x79, 0x81, 0x3c, 0x49, 0xd0, 0xd5, 0x24, 0xa7, 0x39, 0x3f, 0xd2, 0x1a, 0x0b, 0xdf, 0xba, 0xab, 0x45, 0xbf, 0x44, 0x3f, 0x49, 0xb1, 0xb3, 0x24, 0xb5, 0x14, 0x25, 0xfb, 0x82, 0xc3, 0xbd, 0x71,
0xaa, 0xab, 0xab, 0xeb, 0xe7, 0xab, 0x22, 0xa1, 0x7d, 0xca, 0x22, 0xca, 0x77, 0x53, 0x9e, 0xc8, 0x67, 0x66, 0x67, 0x67, 0xe7, 0xcf, 0x6f, 0x66, 0x25, 0xe8, 0x1e, 0xb3, 0x88, 0xf2, 0xad, 0x94,
0x84, 0xb4, 0x70, 0xe3, 0xa5, 0x27, 0xce, 0xd7, 0xb0, 0xfe, 0x26, 0x49, 0xce, 0xb3, 0xf4, 0x39, 0x27, 0x32, 0x21, 0x1d, 0x5c, 0x78, 0xe9, 0x91, 0xf3, 0x25, 0xdc, 0x7d, 0x9d, 0x24, 0xa7, 0x59,
0xe3, 0x34, 0x90, 0x09, 0xbf, 0x3e, 0x88, 0x25, 0xbf, 0x76, 0xe9, 0x9f, 0x33, 0x2a, 0x24, 0xd9, 0xfa, 0x9c, 0x71, 0x1a, 0xc8, 0x84, 0x5f, 0xee, 0xc5, 0x92, 0x5f, 0xba, 0xf4, 0x4f, 0x19, 0x15,
0x00, 0x3b, 0x2c, 0x18, 0x3d, 0x6b, 0xdb, 0xda, 0xb1, 0xdd, 0x11, 0x81, 0x10, 0x98, 0x8f, 0xfd, 0x92, 0xdc, 0x03, 0x3b, 0x2c, 0x18, 0x03, 0x6b, 0xc3, 0xda, 0xb4, 0xdd, 0x09, 0x81, 0x10, 0x58,
0x21, 0xed, 0xd5, 0x90, 0x81, 0x6b, 0xe7, 0x00, 0x36, 0xa6, 0x2b, 0x14, 0x69, 0x12, 0x0b, 0x4a, 0x8c, 0xfd, 0x31, 0x1d, 0x34, 0x90, 0x81, 0xdf, 0xce, 0x1e, 0xdc, 0x9b, 0xad, 0x50, 0xa4, 0x49,
0x1e, 0x42, 0x83, 0x2a, 0x02, 0x6a, 0x6b, 0xef, 0x7d, 0xb2, 0x5b, 0x98, 0xb2, 0xab, 0xe5, 0x34, 0x2c, 0x28, 0x79, 0x04, 0x2d, 0xaa, 0x08, 0xa8, 0xad, 0xbb, 0xfd, 0xc1, 0x56, 0x61, 0xca, 0x96,
0xd7, 0xf9, 0x87, 0x05, 0xe4, 0x0d, 0x13, 0x52, 0x11, 0x19, 0x15, 0xdf, 0xcf, 0x9e, 0xbb, 0xd0, 0x96, 0xd3, 0x5c, 0xe7, 0x1f, 0x16, 0x90, 0xd7, 0x4c, 0x48, 0x45, 0x64, 0x54, 0x7c, 0x37, 0x7b,
0x4c, 0x39, 0x3d, 0x65, 0x57, 0xb9, 0x45, 0xf9, 0x8e, 0x3c, 0x86, 0x25, 0x21, 0x7d, 0x2e, 0x5f, 0x6e, 0x41, 0x3b, 0xe5, 0xf4, 0x98, 0x5d, 0xe4, 0x16, 0xe5, 0x2b, 0xf2, 0x04, 0x56, 0x84, 0xf4,
0xf0, 0x64, 0xf8, 0x82, 0x45, 0xf4, 0x50, 0x19, 0x5d, 0x47, 0x91, 0x49, 0x06, 0xd9, 0x05, 0xc2, 0xb9, 0x7c, 0xc1, 0x93, 0xf1, 0x0b, 0x16, 0xd1, 0x7d, 0x65, 0x74, 0x13, 0x45, 0xea, 0x0c, 0xb2,
0xe2, 0x20, 0xca, 0x04, 0xbb, 0xa0, 0xc7, 0x05, 0xb7, 0x37, 0xbf, 0x6d, 0xed, 0xb4, 0xdc, 0x29, 0x05, 0x84, 0xc5, 0x41, 0x94, 0x09, 0x76, 0x46, 0x0f, 0x0b, 0xee, 0x60, 0x71, 0xc3, 0xda, 0xec,
0x1c, 0xb2, 0x02, 0x8d, 0x88, 0x0d, 0x99, 0xec, 0x35, 0xb6, 0xad, 0x9d, 0xae, 0xab, 0x37, 0xce, 0xb8, 0x33, 0x38, 0x64, 0x0d, 0x5a, 0x11, 0x1b, 0x33, 0x39, 0x68, 0x6d, 0x58, 0x9b, 0x7d, 0x57,
0x2f, 0x61, 0xb9, 0x62, 0xff, 0x87, 0x3d, 0xff, 0xaf, 0x35, 0x68, 0x20, 0xa1, 0xf4, 0xb1, 0x35, 0x2f, 0x9c, 0x5f, 0xc2, 0x6a, 0xc5, 0xfe, 0xf7, 0xbb, 0xfe, 0x5f, 0x1a, 0xd0, 0x42, 0x42, 0xe9,
0xf2, 0x31, 0xf9, 0x14, 0x3a, 0x4c, 0x78, 0x23, 0x47, 0xd4, 0xd0, 0xb6, 0x36, 0x13, 0xa5, 0xcf, 0x63, 0x6b, 0xe2, 0x63, 0xf2, 0x21, 0xf4, 0x98, 0xf0, 0x26, 0x8e, 0x68, 0xa0, 0x6d, 0x5d, 0x26,
0xc9, 0x23, 0x68, 0x06, 0x67, 0x59, 0x7c, 0x2e, 0x7a, 0xf5, 0xed, 0xfa, 0x4e, 0x7b, 0x6f, 0x79, 0x4a, 0x9f, 0x93, 0xc7, 0xd0, 0x0e, 0x4e, 0xb2, 0xf8, 0x54, 0x0c, 0x9a, 0x1b, 0xcd, 0xcd, 0xee,
0x74, 0x91, 0x7a, 0xe8, 0xbe, 0xe2, 0xb9, 0xb9, 0x08, 0xf9, 0x02, 0xc0, 0x97, 0x92, 0xb3, 0x93, 0xf6, 0xea, 0xe4, 0x20, 0x75, 0xd1, 0x5d, 0xc5, 0x73, 0x73, 0x11, 0xf2, 0x29, 0x80, 0x2f, 0x25,
0x4c, 0x52, 0x81, 0x2f, 0x6d, 0xef, 0xf5, 0x8c, 0x03, 0x99, 0xa0, 0x4f, 0x4b, 0xbe, 0x6b, 0xc8, 0x67, 0x47, 0x99, 0xa4, 0x02, 0x6f, 0xda, 0xdd, 0x1e, 0x18, 0x1b, 0x32, 0x41, 0x77, 0x4a, 0xbe,
0x92, 0x2f, 0xa1, 0x45, 0xaf, 0x24, 0x8d, 0x43, 0x1a, 0xf6, 0x1a, 0x78, 0xd1, 0xe6, 0xd8, 0x8b, 0x6b, 0xc8, 0x92, 0xcf, 0xa0, 0x43, 0x2f, 0x24, 0x8d, 0x43, 0x1a, 0x0e, 0x5a, 0x78, 0xd0, 0xfa,
0x76, 0x0f, 0x72, 0xbe, 0x7e, 0x5f, 0x29, 0xde, 0xff, 0x0a, 0xba, 0x15, 0x16, 0x59, 0x84, 0xfa, 0xd4, 0x8d, 0xb6, 0xf6, 0x72, 0xbe, 0xbe, 0x5f, 0x29, 0x3e, 0xfc, 0x1c, 0xfa, 0x15, 0x16, 0x59,
0x39, 0x2d, 0xa2, 0xaa, 0x96, 0xca, 0xb3, 0x17, 0x7e, 0x94, 0xe9, 0x04, 0xeb, 0xb8, 0x7a, 0xf3, 0x86, 0xe6, 0x29, 0x2d, 0xa2, 0xaa, 0x3e, 0x95, 0x67, 0xcf, 0xfc, 0x28, 0xd3, 0x09, 0xd6, 0x73,
0x8b, 0xda, 0x17, 0x96, 0xf3, 0x1c, 0xec, 0x17, 0x59, 0x14, 0x95, 0x07, 0x43, 0xc6, 0x8b, 0x83, 0xf5, 0xe2, 0x17, 0x8d, 0x4f, 0x2d, 0xe7, 0x39, 0xd8, 0x2f, 0xb2, 0x28, 0x2a, 0x37, 0x86, 0x8c,
0x21, 0xe3, 0x23, 0x2f, 0xd7, 0x6e, 0xf4, 0xf2, 0xdf, 0x2d, 0x58, 0x3a, 0xb8, 0xa0, 0xb1, 0x3c, 0x17, 0x1b, 0x43, 0xc6, 0x27, 0x5e, 0x6e, 0x5c, 0xe9, 0xe5, 0xbf, 0x5b, 0xb0, 0xb2, 0x77, 0x46,
0x4c, 0x24, 0x3b, 0x65, 0x81, 0x2f, 0x59, 0x12, 0x93, 0xc7, 0x60, 0x27, 0x51, 0xe8, 0xdd, 0x18, 0x63, 0xb9, 0x9f, 0x48, 0x76, 0xcc, 0x02, 0x5f, 0xb2, 0x24, 0x26, 0x4f, 0xc0, 0x4e, 0xa2, 0xd0,
0xa6, 0x56, 0x12, 0xe5, 0x56, 0x3f, 0x06, 0x3b, 0xa6, 0x97, 0xde, 0x8d, 0xd7, 0xb5, 0x62, 0x7a, 0xbb, 0x32, 0x4c, 0x9d, 0x24, 0xca, 0xad, 0x7e, 0x02, 0x76, 0x4c, 0xcf, 0xbd, 0x2b, 0x8f, 0xeb,
0xa9, 0xa5, 0x1f, 0x40, 0x37, 0xa4, 0x11, 0x95, 0xd4, 0x2b, 0xa3, 0xa3, 0x42, 0xd7, 0xd1, 0xc4, 0xc4, 0xf4, 0x5c, 0x4b, 0x3f, 0x84, 0x7e, 0x48, 0x23, 0x2a, 0xa9, 0x57, 0x46, 0x47, 0x85, 0xae,
0x7d, 0x1d, 0x8e, 0xcf, 0xe0, 0x13, 0xa5, 0x32, 0xf5, 0x39, 0x8d, 0xa5, 0x97, 0xfa, 0xf2, 0x0c, 0xa7, 0x89, 0xbb, 0x3a, 0x1c, 0x1f, 0xc1, 0x07, 0x4a, 0x65, 0xea, 0x73, 0x1a, 0x4b, 0x2f, 0xf5,
0x63, 0x62, 0xbb, 0xdd, 0x98, 0x5e, 0x1e, 0x21, 0xf5, 0xc8, 0x97, 0x67, 0xce, 0xdf, 0x6a, 0x60, 0xe5, 0x09, 0xc6, 0xc4, 0x76, 0xfb, 0x31, 0x3d, 0x3f, 0x40, 0xea, 0x81, 0x2f, 0x4f, 0x9c, 0xbf,
0x97, 0xc1, 0x24, 0xf7, 0x60, 0x41, 0x5d, 0xeb, 0xb1, 0x30, 0xf7, 0x44, 0x53, 0x6d, 0x5f, 0x85, 0x35, 0xc0, 0x2e, 0x83, 0x49, 0x6e, 0xc3, 0x92, 0x3a, 0xd6, 0x63, 0x61, 0xee, 0x89, 0xb6, 0x5a,
0xaa, 0x2a, 0x92, 0xd3, 0x53, 0x41, 0x25, 0x9a, 0x57, 0x77, 0xf3, 0x9d, 0xca, 0x2c, 0xc1, 0xbe, 0xbe, 0x0c, 0x55, 0x55, 0x24, 0xc7, 0xc7, 0x82, 0x4a, 0x34, 0xaf, 0xe9, 0xe6, 0x2b, 0x95, 0x59,
0xd5, 0x85, 0x30, 0xef, 0xe2, 0x5a, 0x79, 0x7c, 0x28, 0xd9, 0x90, 0xe2, 0x85, 0x75, 0x57, 0x6f, 0x82, 0x7d, 0xa3, 0x0b, 0x61, 0xd1, 0xc5, 0x6f, 0xe5, 0xf1, 0xb1, 0x64, 0x63, 0x8a, 0x07, 0x36,
0xc8, 0x32, 0x34, 0xa8, 0x27, 0xfd, 0x01, 0x66, 0xb8, 0xed, 0xce, 0xd3, 0x77, 0xfe, 0x80, 0xfc, 0x5d, 0xbd, 0x20, 0xab, 0xd0, 0xa2, 0x9e, 0xf4, 0x47, 0x98, 0xe1, 0xb6, 0xbb, 0x48, 0xdf, 0xfa,
0x08, 0xee, 0x88, 0x24, 0xe3, 0x01, 0xf5, 0x8a, 0x6b, 0x9b, 0xc8, 0xed, 0x68, 0xea, 0x0b, 0x7d, 0x23, 0xf2, 0x23, 0xb8, 0x21, 0x92, 0x8c, 0x07, 0xd4, 0x2b, 0x8e, 0x6d, 0x23, 0xb7, 0xa7, 0xa9,
0xb9, 0x03, 0xf5, 0x53, 0x16, 0xf6, 0x16, 0xd0, 0x31, 0x8b, 0xd5, 0x24, 0x7c, 0x15, 0xba, 0x8a, 0x2f, 0xf4, 0xe1, 0x0e, 0x34, 0x8f, 0x59, 0x38, 0x58, 0x42, 0xc7, 0x2c, 0x57, 0x93, 0xf0, 0x65,
0x49, 0x7e, 0x0a, 0x50, 0x6a, 0x0a, 0x7b, 0xad, 0x19, 0xa2, 0x76, 0xa1, 0x37, 0x24, 0x9b, 0x00, 0xe8, 0x2a, 0x26, 0xf9, 0x09, 0x40, 0xa9, 0x29, 0x1c, 0x74, 0xe6, 0x88, 0xda, 0x85, 0xde, 0x90,
0x01, 0x4b, 0xcf, 0x28, 0xf7, 0x54, 0xc2, 0xd8, 0x98, 0x1c, 0xb6, 0xa6, 0xbc, 0xa6, 0xd7, 0x8a, 0xac, 0x03, 0x04, 0x2c, 0x3d, 0xa1, 0xdc, 0x53, 0x09, 0x63, 0x63, 0x72, 0xd8, 0x9a, 0xf2, 0x8a,
0xcd, 0x84, 0x37, 0xf8, 0x96, 0xa5, 0x29, 0x0d, 0x7b, 0x80, 0x1e, 0xb6, 0x99, 0x78, 0xa9, 0x09, 0x5e, 0x2a, 0x36, 0x13, 0xde, 0xe8, 0x1b, 0x96, 0xa6, 0x34, 0x1c, 0x00, 0x7a, 0xd8, 0x66, 0xe2,
0xce, 0x6f, 0xa1, 0x99, 0x1b, 0xb7, 0x0e, 0xf6, 0x45, 0x12, 0x65, 0xc3, 0xd2, 0x69, 0x5d, 0xb7, 0x0b, 0x4d, 0x70, 0xbe, 0x86, 0x76, 0x6e, 0xdc, 0x5d, 0xb0, 0xcf, 0x92, 0x28, 0x1b, 0x97, 0x4e,
0xa5, 0x09, 0xaf, 0x42, 0xb2, 0x06, 0x88, 0x92, 0x78, 0x45, 0x0d, 0x5d, 0x84, 0xfe, 0x55, 0x17, 0xeb, 0xbb, 0x1d, 0x4d, 0x78, 0x19, 0x92, 0x3b, 0x80, 0x28, 0x89, 0x47, 0x34, 0xd0, 0x45, 0xe8,
0xdc, 0x85, 0x66, 0x90, 0x24, 0xe7, 0x4c, 0xfb, 0x6e, 0xc1, 0xcd, 0x77, 0xce, 0x77, 0x75, 0xb8, 0x5f, 0x75, 0xc0, 0x2d, 0x68, 0x07, 0x49, 0x72, 0xca, 0xb4, 0xef, 0x96, 0xdc, 0x7c, 0xe5, 0x7c,
0x53, 0x2d, 0x16, 0x75, 0x05, 0x6a, 0x41, 0x4f, 0x5b, 0xa8, 0x06, 0xd5, 0x1e, 0x57, 0xbc, 0x5d, 0xdb, 0x84, 0x1b, 0xd5, 0x62, 0x51, 0x47, 0xa0, 0x16, 0xf4, 0xb4, 0x85, 0x6a, 0x50, 0xed, 0x61,
0x33, 0xbd, 0x5d, 0x1c, 0x19, 0x26, 0xa1, 0xbe, 0xa0, 0xab, 0x8f, 0xbc, 0x4d, 0x42, 0xaa, 0x72, 0xc5, 0xdb, 0x0d, 0xd3, 0xdb, 0xc5, 0x96, 0x71, 0x12, 0xea, 0x03, 0xfa, 0x7a, 0xcb, 0x9b, 0x24,
0x3d, 0x63, 0x21, 0x86, 0xa7, 0xeb, 0xaa, 0xa5, 0xa2, 0x0c, 0x58, 0x98, 0x83, 0x8f, 0x5a, 0xa2, 0xa4, 0x2a, 0xd7, 0x33, 0x16, 0x62, 0x78, 0xfa, 0xae, 0xfa, 0x54, 0x94, 0x11, 0x0b, 0x73, 0xf0,
0x79, 0x1c, 0xf5, 0x36, 0x75, 0xc0, 0xf5, 0x4e, 0x05, 0x7c, 0xa8, 0xa8, 0x0b, 0x3a, 0x8a, 0x6a, 0x51, 0x9f, 0x68, 0x1e, 0x47, 0xbd, 0x6d, 0x1d, 0x70, 0xbd, 0x52, 0x01, 0x1f, 0x2b, 0xea, 0x92,
0x4d, 0xb6, 0xa1, 0xcd, 0x69, 0x1a, 0xe5, 0xb9, 0x8f, 0xce, 0xb7, 0x5d, 0x93, 0x44, 0xb6, 0x00, 0x8e, 0xa2, 0xfa, 0x26, 0x1b, 0xd0, 0xe5, 0x34, 0x8d, 0xf2, 0xdc, 0x47, 0xe7, 0xdb, 0xae, 0x49,
0x82, 0x24, 0x8a, 0x68, 0x80, 0x02, 0x36, 0x0a, 0x18, 0x14, 0x95, 0x77, 0x52, 0x46, 0x9e, 0xa0, 0x22, 0xf7, 0x01, 0x82, 0x24, 0x8a, 0x68, 0x80, 0x02, 0x36, 0x0a, 0x18, 0x14, 0x95, 0x77, 0x52,
0x01, 0xba, 0xba, 0xe1, 0x36, 0xa5, 0x8c, 0x8e, 0x69, 0xa0, 0xde, 0x91, 0x09, 0xca, 0x3d, 0x84, 0x46, 0x9e, 0xa0, 0x01, 0xba, 0xba, 0xe5, 0xb6, 0xa5, 0x8c, 0x0e, 0x69, 0xa0, 0xee, 0x91, 0x09,
0xaf, 0x36, 0x9e, 0x6b, 0x29, 0x02, 0x82, 0xec, 0x26, 0xc0, 0x80, 0x27, 0x59, 0xaa, 0xb9, 0x9d, 0xca, 0x3d, 0x84, 0xaf, 0x2e, 0xee, 0xeb, 0x28, 0x02, 0x82, 0xec, 0x3a, 0xc0, 0x88, 0x27, 0x59,
0xed, 0xba, 0x42, 0x72, 0xa4, 0x20, 0xfb, 0x21, 0xdc, 0x11, 0xd7, 0xc3, 0x88, 0xc5, 0xe7, 0x9e, 0xaa, 0xb9, 0xbd, 0x8d, 0xa6, 0x42, 0x72, 0xa4, 0x20, 0xfb, 0x11, 0xdc, 0x10, 0x97, 0xe3, 0x88,
0xf4, 0xf9, 0x80, 0xca, 0x5e, 0x57, 0x57, 0x40, 0x4e, 0x7d, 0x87, 0x44, 0xf5, 0xf6, 0x61, 0xf8, 0xc5, 0xa7, 0x9e, 0xf4, 0xf9, 0x88, 0xca, 0x41, 0x5f, 0x57, 0x40, 0x4e, 0x7d, 0x8b, 0x44, 0x75,
0xf3, 0xde, 0x1d, 0xcc, 0x00, 0xb5, 0x74, 0x52, 0x20, 0xfb, 0x9c, 0xfa, 0x92, 0x7e, 0x40, 0x1b, 0xf7, 0x71, 0xf8, 0xf3, 0xc1, 0x0d, 0xcc, 0x00, 0xf5, 0xe9, 0xa4, 0x40, 0x76, 0x39, 0xf5, 0x25,
0xfb, 0x7e, 0x68, 0x41, 0x56, 0xa1, 0x99, 0x78, 0xf4, 0x2a, 0x88, 0xf2, 0xa2, 0x6d, 0x24, 0x07, 0x7d, 0x8f, 0x36, 0xf6, 0xdd, 0xd0, 0x82, 0xdc, 0x84, 0x76, 0xe2, 0xd1, 0x8b, 0x20, 0xca, 0x8b,
0x57, 0x41, 0xe4, 0x3c, 0x82, 0xe5, 0xca, 0x8d, 0x39, 0xd0, 0xaf, 0x40, 0x83, 0x72, 0x9e, 0x14, 0xb6, 0x95, 0xec, 0x5d, 0x04, 0x91, 0xf3, 0x18, 0x56, 0x2b, 0x27, 0xe6, 0x40, 0xbf, 0x06, 0x2d,
0xb0, 0xa4, 0x37, 0xce, 0xef, 0x80, 0xbc, 0x4f, 0xc3, 0x8f, 0x61, 0x9e, 0xb3, 0x0a, 0xcb, 0x15, 0xca, 0x79, 0x52, 0xc0, 0x92, 0x5e, 0x38, 0xbf, 0x05, 0xf2, 0x2e, 0x0d, 0x7f, 0x08, 0xf3, 0x9c,
0xd5, 0xda, 0x0e, 0xe7, 0x3b, 0x0b, 0x56, 0x9e, 0xa6, 0x29, 0x8d, 0xc3, 0x77, 0xc9, 0x07, 0x5c, 0x9b, 0xb0, 0x5a, 0x51, 0xad, 0xed, 0x70, 0xbe, 0xb5, 0x60, 0x6d, 0x27, 0x4d, 0x69, 0x1c, 0xbe,
0xba, 0x09, 0x80, 0x6a, 0x3d, 0xa3, 0xc1, 0xdb, 0x48, 0xc1, 0xf8, 0x7c, 0x48, 0x7b, 0x71, 0xee, 0x4d, 0xde, 0xe3, 0xd0, 0x75, 0x00, 0x54, 0xeb, 0x19, 0x0d, 0xde, 0x46, 0x0a, 0xc6, 0xe7, 0x7d,
0xc1, 0xea, 0x98, 0x05, 0xb9, 0x6d, 0xff, 0xb2, 0x80, 0x3c, 0x47, 0xe4, 0xfb, 0x61, 0x43, 0x87, 0xda, 0x8b, 0x73, 0x1b, 0x6e, 0x4e, 0x59, 0x90, 0xdb, 0xf6, 0x2f, 0x0b, 0xc8, 0x73, 0x44, 0xbe,
0xc2, 0x22, 0xd5, 0x10, 0x35, 0xb2, 0x86, 0xbe, 0xf4, 0xf3, 0x76, 0xdd, 0x61, 0x42, 0xeb, 0x7f, 0xef, 0x37, 0x74, 0x28, 0x2c, 0x52, 0x0d, 0x51, 0x23, 0x6b, 0xe8, 0x4b, 0x3f, 0x6f, 0xd7, 0x3d,
0xee, 0x4b, 0x3f, 0x6f, 0x9b, 0x9c, 0x06, 0x19, 0x57, 0x1d, 0x1c, 0x4b, 0x06, 0xdb, 0xa6, 0x5b, 0x26, 0xb4, 0xfe, 0xe7, 0xbe, 0xf4, 0xf3, 0xb6, 0xc9, 0x69, 0x90, 0x71, 0xd5, 0xc1, 0xb1, 0x64,
0x90, 0xc8, 0xe7, 0x70, 0x97, 0x0d, 0xe2, 0x84, 0xd3, 0x91, 0x98, 0xa7, 0xc3, 0xd8, 0x44, 0xe1, 0xb0, 0x6d, 0xba, 0x05, 0x89, 0x7c, 0x02, 0xb7, 0xd8, 0x28, 0x4e, 0x38, 0x9d, 0x88, 0x79, 0x3a,
0x15, 0xcd, 0x2d, 0x0f, 0x1c, 0x60, 0x54, 0x1f, 0xc1, 0x72, 0xe5, 0x19, 0x37, 0xa6, 0xc0, 0x5f, 0x8c, 0x6d, 0x14, 0x5e, 0xd3, 0xdc, 0x72, 0xc3, 0x1e, 0x46, 0xf5, 0x31, 0xac, 0x56, 0xae, 0x71,
0x2c, 0xe8, 0x3d, 0x95, 0xc9, 0x90, 0x05, 0x2e, 0x55, 0xc6, 0x57, 0x9e, 0xfe, 0x00, 0xba, 0xaa, 0x65, 0x0a, 0xfc, 0xd9, 0x82, 0xc1, 0x8e, 0x4c, 0xc6, 0x2c, 0x70, 0xa9, 0x32, 0xbe, 0x72, 0xf5,
0xf7, 0x8c, 0x3f, 0xbf, 0x93, 0x44, 0xe1, 0xa8, 0xb7, 0xaf, 0x81, 0x6a, 0x3f, 0x66, 0x64, 0x16, 0x87, 0xd0, 0x57, 0xbd, 0x67, 0xfa, 0xfa, 0xbd, 0x24, 0x0a, 0x27, 0xbd, 0xfd, 0x0e, 0xa8, 0xf6,
0x92, 0x28, 0xc4, 0xb8, 0x3c, 0x00, 0xd5, 0x23, 0x8c, 0xf3, 0x7a, 0xca, 0xe9, 0xc4, 0xf4, 0xb2, 0x63, 0x46, 0x66, 0x29, 0x89, 0x42, 0x8c, 0xcb, 0x43, 0x50, 0x3d, 0xc2, 0xd8, 0xaf, 0xa7, 0x9c,
0x72, 0x5e, 0x09, 0xe1, 0x79, 0xdd, 0x58, 0x16, 0x62, 0x7a, 0xa9, 0xce, 0x3b, 0xeb, 0xb0, 0x36, 0x5e, 0x4c, 0xcf, 0x2b, 0xfb, 0x95, 0x10, 0xee, 0xd7, 0x8d, 0x65, 0x29, 0xa6, 0xe7, 0x6a, 0xbf,
0xc5, 0xb6, 0x3c, 0x5c, 0xff, 0xb6, 0x60, 0xf9, 0xa9, 0x10, 0x6c, 0x10, 0x7f, 0x83, 0x20, 0x59, 0x73, 0x17, 0xee, 0xcc, 0xb0, 0x2d, 0x0f, 0xd7, 0xbf, 0x2d, 0x58, 0xdd, 0x11, 0x82, 0x8d, 0xe2,
0x18, 0xbd, 0x02, 0x8d, 0x20, 0xc9, 0x62, 0x89, 0xc6, 0x36, 0x5c, 0xbd, 0x19, 0xc3, 0x8d, 0xda, 0xaf, 0x10, 0x24, 0x0b, 0xa3, 0xd7, 0xa0, 0x15, 0x24, 0x59, 0x2c, 0xd1, 0xd8, 0x96, 0xab, 0x17,
0x04, 0x6e, 0x8c, 0x21, 0x4f, 0x7d, 0x12, 0x79, 0x0c, 0x64, 0x99, 0xaf, 0x20, 0xcb, 0x7d, 0x68, 0x53, 0xb8, 0xd1, 0xa8, 0xe1, 0xc6, 0x14, 0xf2, 0x34, 0xeb, 0xc8, 0x63, 0x20, 0xcb, 0x62, 0x05,
0xab, 0x20, 0x7b, 0x01, 0x8d, 0x25, 0xe5, 0x79, 0x57, 0x02, 0x45, 0xda, 0x47, 0x8a, 0x12, 0x30, 0x59, 0x1e, 0x40, 0x57, 0x05, 0xd9, 0x0b, 0x68, 0x2c, 0x29, 0xcf, 0xbb, 0x12, 0x28, 0xd2, 0x2e,
0xbb, 0xa7, 0x6e, 0x4c, 0x90, 0x8e, 0x5a, 0xe7, 0xff, 0x54, 0x55, 0x54, 0x9e, 0x92, 0xc7, 0x6c, 0x52, 0x94, 0x80, 0xd9, 0x3d, 0x75, 0x63, 0x82, 0x74, 0xd2, 0x3a, 0xff, 0xa7, 0xaa, 0xa2, 0x72,
0x66, 0x17, 0x55, 0xc0, 0xcb, 0xa3, 0xfc, 0x1d, 0x6a, 0xa9, 0x4a, 0x24, 0xcd, 0x4e, 0x22, 0x16, 0x95, 0x3c, 0x66, 0x73, 0xbb, 0xa8, 0x02, 0x5e, 0x1e, 0xe5, 0xf7, 0x50, 0x9f, 0xaa, 0x44, 0xd2,
0x78, 0x8a, 0xa1, 0xed, 0xb7, 0x35, 0xe5, 0x3d, 0x8f, 0x46, 0x5e, 0x99, 0x37, 0xbd, 0x42, 0x60, 0xec, 0x28, 0x62, 0x81, 0xa7, 0x18, 0xda, 0x7e, 0x5b, 0x53, 0xde, 0xf1, 0x68, 0xe2, 0x95, 0x45,
0xde, 0xcf, 0xe4, 0x59, 0xd1, 0x49, 0xd5, 0x7a, 0xcc, 0x53, 0xcd, 0xdb, 0x3c, 0xb5, 0x30, 0xe9, 0xd3, 0x2b, 0x04, 0x16, 0xfd, 0x4c, 0x9e, 0x14, 0x9d, 0x54, 0x7d, 0x4f, 0x79, 0xaa, 0x7d, 0x9d,
0xa9, 0x32, 0xd3, 0x5a, 0x66, 0xa6, 0x7d, 0x0e, 0xcb, 0x7a, 0x14, 0xaf, 0x86, 0x6b, 0x13, 0xa0, 0xa7, 0x96, 0xea, 0x9e, 0x2a, 0x33, 0xad, 0x63, 0x66, 0xda, 0x27, 0xb0, 0xaa, 0x47, 0xf1, 0x6a,
0xec, 0x7a, 0xa2, 0x67, 0x69, 0xe8, 0x2d, 0xda, 0x9e, 0x70, 0x7e, 0x05, 0xf6, 0x9b, 0x44, 0xeb, 0xb8, 0xd6, 0x01, 0xca, 0xae, 0x27, 0x06, 0x96, 0x86, 0xde, 0xa2, 0xed, 0x09, 0xe7, 0x57, 0x60,
0x15, 0xe4, 0x09, 0xd8, 0x51, 0xb1, 0x41, 0xd1, 0xf6, 0x1e, 0x19, 0x95, 0x7a, 0x21, 0xe7, 0x8e, 0xbf, 0x4e, 0xb4, 0x5e, 0x41, 0x9e, 0x82, 0x1d, 0x15, 0x0b, 0x14, 0xed, 0x6e, 0x93, 0x49, 0xa9,
0x84, 0x9c, 0xaf, 0xa0, 0x55, 0x90, 0x0b, 0x9f, 0x59, 0xb3, 0x7c, 0x56, 0x1b, 0xf3, 0x99, 0xf3, 0x17, 0x72, 0xee, 0x44, 0xc8, 0xf9, 0x1c, 0x3a, 0x05, 0xb9, 0xf0, 0x99, 0x35, 0xcf, 0x67, 0x8d,
0x4f, 0x0b, 0x56, 0xaa, 0x26, 0xe7, 0x61, 0x79, 0x0f, 0xdd, 0xf2, 0x0a, 0x6f, 0xe8, 0xa7, 0xb9, 0x29, 0x9f, 0x39, 0xff, 0xb4, 0x60, 0xad, 0x6a, 0x72, 0x1e, 0x96, 0x77, 0xd0, 0x2f, 0x8f, 0xf0,
0x2d, 0x4f, 0x4c, 0x5b, 0x26, 0x8f, 0x95, 0x06, 0x8a, 0xb7, 0x7e, 0xaa, 0x73, 0xb9, 0x13, 0x19, 0xc6, 0x7e, 0x9a, 0xdb, 0xf2, 0xd4, 0xb4, 0xa5, 0xbe, 0xad, 0x34, 0x50, 0xbc, 0xf1, 0x53, 0x9d,
0xa4, 0xfe, 0x3b, 0x58, 0x9a, 0x10, 0x99, 0x32, 0x87, 0xfe, 0xc4, 0x9c, 0x43, 0x2b, 0x60, 0x57, 0xcb, 0xbd, 0xc8, 0x20, 0x0d, 0xdf, 0xc2, 0x4a, 0x4d, 0x64, 0xc6, 0x1c, 0xfa, 0x63, 0x73, 0x0e,
0x9e, 0x36, 0x87, 0xd3, 0x2f, 0xe1, 0x9e, 0x86, 0x83, 0xfd, 0x32, 0x86, 0x85, 0xef, 0xab, 0xa1, 0xad, 0x80, 0x5d, 0xb9, 0xdb, 0x1c, 0x4e, 0x3f, 0x83, 0xdb, 0x1a, 0x0e, 0x76, 0xcb, 0x18, 0x16,
0xb6, 0xc6, 0x43, 0xed, 0xf4, 0xa1, 0x37, 0x79, 0x34, 0x2f, 0xbf, 0x01, 0x2c, 0x1d, 0x4b, 0x5f, 0xbe, 0xaf, 0x86, 0xda, 0x9a, 0x0e, 0xb5, 0x33, 0x84, 0x41, 0x7d, 0x6b, 0x5e, 0x7e, 0x23, 0x58,
0x32, 0x21, 0x59, 0x50, 0x7e, 0x10, 0x8d, 0xe5, 0x86, 0x75, 0x5b, 0xff, 0x9e, 0xac, 0xc3, 0x45, 0x39, 0x94, 0xbe, 0x64, 0x42, 0xb2, 0xa0, 0x7c, 0x10, 0x4d, 0xe5, 0x86, 0x75, 0x5d, 0xff, 0xae,
0xa8, 0x4b, 0x59, 0xe4, 0xaf, 0x5a, 0xaa, 0x28, 0x10, 0xf3, 0xa6, 0x3c, 0x06, 0x1f, 0xe1, 0x2a, 0xd7, 0xe1, 0x32, 0x34, 0xa5, 0x2c, 0xf2, 0x57, 0x7d, 0xaa, 0x28, 0x10, 0xf3, 0xa4, 0x3c, 0x06,
0x95, 0x0f, 0x32, 0x91, 0x7e, 0xa4, 0xe7, 0xa3, 0x79, 0x9c, 0x8f, 0x6c, 0xa4, 0xe0, 0x80, 0xa4, 0x3f, 0xc0, 0x51, 0x2a, 0x1f, 0x64, 0x22, 0xfd, 0x48, 0xcf, 0x47, 0x8b, 0x38, 0x1f, 0xd9, 0x48,
0x47, 0x88, 0x50, 0x73, 0x1b, 0x7a, 0x7a, 0x52, 0x04, 0x64, 0x6e, 0x02, 0x60, 0xa9, 0xea, 0x2a, 0xc1, 0x01, 0x49, 0x8f, 0x10, 0xa1, 0xe6, 0xb6, 0xf4, 0xf4, 0xa4, 0x08, 0xc8, 0x5c, 0x07, 0xc0,
0x6b, 0xea, 0xb3, 0x8a, 0xb2, 0xaf, 0x08, 0xce, 0x16, 0x6c, 0xbc, 0xa4, 0x52, 0x75, 0x23, 0xbe, 0x52, 0xd5, 0x55, 0xd6, 0xd6, 0x7b, 0x15, 0x65, 0x57, 0x11, 0x9c, 0xfb, 0x70, 0xef, 0x0b, 0x2a,
0x9f, 0xc4, 0xa7, 0x6c, 0x90, 0x71, 0xdf, 0x08, 0x85, 0xf3, 0x1f, 0x0b, 0x36, 0x67, 0x08, 0xe4, 0x55, 0x37, 0xe2, 0xbb, 0x49, 0x7c, 0xcc, 0x46, 0x19, 0xf7, 0x8d, 0x50, 0x38, 0xff, 0xb1, 0x60,
0x0f, 0xee, 0xc1, 0xc2, 0xd0, 0x17, 0x92, 0xf2, 0xa2, 0x4a, 0x8a, 0xed, 0xb8, 0x2b, 0x6a, 0xb7, 0x7d, 0x8e, 0x40, 0x7e, 0xe1, 0x01, 0x2c, 0x8d, 0x7d, 0x21, 0x29, 0x2f, 0xaa, 0xa4, 0x58, 0x4e,
0xb9, 0xa2, 0x3e, 0xe1, 0x8a, 0x55, 0x68, 0x0e, 0xfd, 0x2b, 0x6f, 0x78, 0x92, 0x8f, 0x72, 0x8d, 0xbb, 0xa2, 0x71, 0x9d, 0x2b, 0x9a, 0x35, 0x57, 0xdc, 0x84, 0xf6, 0xd8, 0xbf, 0xf0, 0xc6, 0x47,
0xa1, 0x7f, 0xf5, 0xf6, 0x04, 0x91, 0x8d, 0x71, 0xef, 0x24, 0x0b, 0xce, 0xa9, 0x14, 0x25, 0xb2, 0xf9, 0x28, 0xd7, 0x1a, 0xfb, 0x17, 0x6f, 0x8e, 0x10, 0xd9, 0x18, 0xf7, 0x8e, 0xb2, 0xe0, 0x94,
0x31, 0xfe, 0x4c, 0x53, 0x70, 0xb6, 0xc3, 0x41, 0x17, 0x61, 0xa0, 0xe5, 0xe6, 0x3b, 0xe7, 0x12, 0x4a, 0x51, 0x22, 0x1b, 0xe3, 0xcf, 0x34, 0x05, 0x67, 0x3b, 0x1c, 0x74, 0x11, 0x06, 0x3a, 0x6e,
0x7a, 0xc7, 0xd9, 0x89, 0x08, 0x38, 0x3b, 0xa1, 0x6f, 0xa9, 0xf4, 0x15, 0x18, 0x16, 0x39, 0x72, 0xbe, 0x72, 0xce, 0x61, 0x70, 0x98, 0x1d, 0x89, 0x80, 0xb3, 0x23, 0xfa, 0x86, 0x4a, 0x5f, 0x81,
0x1f, 0xda, 0x41, 0xc4, 0x14, 0x1a, 0x1a, 0x5f, 0x92, 0xa0, 0x49, 0xd8, 0x35, 0x10, 0x2e, 0xe5, 0x61, 0x91, 0x23, 0x0f, 0xa0, 0x1b, 0x44, 0x4c, 0xa1, 0xa1, 0xf1, 0x92, 0x04, 0x4d, 0xc2, 0xae,
0x99, 0x57, 0xf9, 0x78, 0x06, 0x45, 0x3a, 0xd2, 0x1f, 0xd0, 0x6b, 0xd0, 0x12, 0x2c, 0x0e, 0xa8, 0x81, 0x70, 0x29, 0x4f, 0xbc, 0xca, 0xe3, 0x19, 0x14, 0xe9, 0x40, 0x3f, 0xa0, 0xef, 0x40, 0x47,
0x17, 0xeb, 0x2f, 0x96, 0xba, 0xbb, 0x80, 0xfb, 0x43, 0xa1, 0xda, 0xd9, 0xda, 0x94, 0x9b, 0x73, 0xb0, 0x38, 0xa0, 0x5e, 0xac, 0x5f, 0x2c, 0x4d, 0x77, 0x09, 0xd7, 0xfb, 0x42, 0xb5, 0xb3, 0x3b,
0x17, 0xde, 0xdc, 0xca, 0x7f, 0x03, 0x84, 0x5e, 0xa0, 0x5d, 0xc6, 0xf7, 0x57, 0x5e, 0x64, 0xeb, 0x33, 0x4e, 0xce, 0x5d, 0x78, 0x75, 0x2b, 0xff, 0x0d, 0x10, 0x7a, 0x86, 0x76, 0x19, 0xef, 0xaf,
0xc6, 0x98, 0x33, 0xfe, 0x89, 0xe6, 0x2e, 0xd1, 0x89, 0xaf, 0xb6, 0x65, 0x68, 0x48, 0x31, 0xb2, 0xbc, 0xc8, 0xee, 0x1a, 0x63, 0xce, 0xf4, 0x13, 0xcd, 0x5d, 0xa1, 0xb5, 0x57, 0xdb, 0x2a, 0xb4,
0x6f, 0x5e, 0x8a, 0x43, 0xe1, 0xf8, 0x0a, 0x8c, 0x06, 0xba, 0xac, 0x4b, 0x01, 0x6b, 0x24, 0x40, 0xa4, 0x98, 0xd8, 0xb7, 0x28, 0xc5, 0xbe, 0x70, 0x7c, 0x05, 0x46, 0x23, 0x5d, 0xd6, 0xa5, 0x80,
0x1e, 0x03, 0x49, 0x7d, 0x2e, 0x99, 0x52, 0xa1, 0x26, 0x7d, 0xef, 0xcc, 0x17, 0x67, 0x68, 0x41, 0x35, 0x11, 0x20, 0x4f, 0x80, 0xa4, 0x3e, 0x97, 0x4c, 0xa9, 0x50, 0x93, 0xbe, 0x77, 0xe2, 0x8b,
0xc3, 0x5d, 0x2c, 0x39, 0xaf, 0xe9, 0xf5, 0xaf, 0x7d, 0x71, 0xa6, 0xc0, 0x1b, 0x87, 0x8b, 0x3a, 0x13, 0xb4, 0xa0, 0xe5, 0x2e, 0x97, 0x9c, 0x57, 0xf4, 0xf2, 0xd7, 0xbe, 0x38, 0x51, 0xe0, 0x8d,
0xce, 0x9b, 0xb8, 0x76, 0x5e, 0xc2, 0xca, 0x6b, 0x4a, 0xd3, 0xfd, 0x24, 0x8e, 0x69, 0x20, 0x69, 0xc3, 0x45, 0x13, 0xe7, 0x4d, 0xfc, 0x76, 0x28, 0xac, 0xbd, 0xa2, 0x34, 0xdd, 0x4d, 0xe2, 0x98,
0x58, 0x38, 0x7d, 0xda, 0x77, 0xfb, 0x3a, 0xd8, 0x03, 0x9e, 0x06, 0x5e, 0x9a, 0x70, 0xfd, 0x31, 0x06, 0x92, 0x86, 0x85, 0xd3, 0x67, 0xbd, 0xdb, 0xef, 0x82, 0x3d, 0xe2, 0x69, 0xe0, 0xa5, 0x09,
0xd6, 0x75, 0x5b, 0x8a, 0x70, 0x94, 0x70, 0xa9, 0xa6, 0xa4, 0x31, 0x45, 0xda, 0x87, 0x7b, 0xff, 0xd7, 0x8f, 0xb1, 0xbe, 0xdb, 0x51, 0x84, 0x83, 0x84, 0xe3, 0xd4, 0xc3, 0xa9, 0x7e, 0xe3, 0xe8,
0xb5, 0xa1, 0x73, 0x4c, 0xfd, 0x4b, 0x4a, 0x43, 0x4c, 0x56, 0x32, 0x28, 0x40, 0xb2, 0xfa, 0x8b, 0xa9, 0xca, 0x76, 0x27, 0x04, 0x35, 0x43, 0x4d, 0x1d, 0x93, 0xa3, 0xc2, 0x4f, 0x15, 0xc8, 0x07,
0x85, 0x3c, 0x1c, 0x47, 0xc3, 0xa9, 0xff, 0x74, 0xfa, 0x9f, 0xdd, 0x26, 0x96, 0xe3, 0xcd, 0x1c, 0xbe, 0xa4, 0xcf, 0x78, 0x72, 0x4a, 0x79, 0x71, 0xfc, 0x10, 0x3a, 0xc5, 0xe6, 0xdc, 0x84, 0x72,
0x39, 0x84, 0xb6, 0xf1, 0x0f, 0x83, 0x6c, 0x18, 0x07, 0x27, 0x7e, 0xcd, 0xf4, 0x37, 0x67, 0x70, 0xed, 0xfc, 0x17, 0x51, 0xd6, 0xdc, 0x33, 0x19, 0x58, 0x8e, 0x93, 0x2c, 0xd6, 0xad, 0xaf, 0xe3,
0x0b, 0x6d, 0x4f, 0x2c, 0xf2, 0x06, 0xda, 0xc6, 0xa8, 0x6c, 0xea, 0x9b, 0x9c, 0xd9, 0x4d, 0x7d, 0xea, 0x05, 0xd9, 0x33, 0x0d, 0x6b, 0x20, 0xee, 0x7e, 0x3c, 0x85, 0x80, 0x53, 0x8a, 0xb6, 0xdc,
0x53, 0xe6, 0x6b, 0x67, 0x4e, 0x69, 0x33, 0x06, 0x5e, 0x53, 0xdb, 0xe4, 0x88, 0x6d, 0x6a, 0x9b, 0x5c, 0xde, 0xb8, 0xc1, 0xf0, 0x6b, 0xe8, 0x14, 0x64, 0x35, 0xde, 0xa3, 0x23, 0xfc, 0x30, 0xe4,
0x36, 0x25, 0xcf, 0x11, 0x17, 0xba, 0x95, 0x21, 0x95, 0x6c, 0x8d, 0x4e, 0x4c, 0x9b, 0x9f, 0xfb, 0x54, 0x08, 0x2a, 0x72, 0x1b, 0xfb, 0x8a, 0xba, 0x53, 0x10, 0x95, 0x58, 0xb1, 0x3f, 0xaf, 0x72,
0xf7, 0x67, 0xf2, 0x4d, 0x0b, 0x8d, 0xb9, 0xd0, 0xb4, 0x70, 0x72, 0xea, 0x35, 0x2d, 0x9c, 0x32, 0x1d, 0x99, 0x7e, 0x41, 0xc5, 0x4a, 0xdf, 0xfe, 0x2b, 0x40, 0xef, 0x90, 0xfa, 0xe7, 0x94, 0x86,
0x4c, 0x3a, 0x73, 0xe4, 0x0f, 0xb0, 0x34, 0x31, 0x9b, 0x11, 0xc7, 0xb0, 0x62, 0xc6, 0x50, 0xd9, 0x58, 0xcd, 0x64, 0x54, 0x74, 0x91, 0xea, 0x6f, 0x50, 0xe4, 0xd1, 0x74, 0xbb, 0x98, 0xf9, 0xa3,
0x7f, 0x70, 0xa3, 0x4c, 0xa9, 0xff, 0x6b, 0xe8, 0x98, 0x23, 0x11, 0x31, 0x0c, 0x9a, 0x32, 0xf5, 0xd7, 0xf0, 0xa3, 0xeb, 0xc4, 0x72, 0xd7, 0x2f, 0x90, 0x7d, 0xe8, 0x1a, 0x3f, 0xf2, 0x90, 0x7b,
0xf5, 0xb7, 0x66, 0xb1, 0x4d, 0x85, 0x66, 0x57, 0x36, 0x15, 0x4e, 0x99, 0x4b, 0x4c, 0x85, 0xd3, 0xc6, 0xc6, 0xda, 0x6f, 0x57, 0xc3, 0xf5, 0x39, 0xdc, 0x42, 0xdb, 0x53, 0x8b, 0xbc, 0x86, 0xae,
0x9a, 0xb9, 0x33, 0x47, 0x7e, 0x0f, 0x8b, 0xe3, 0xdd, 0x91, 0x7c, 0x3a, 0xee, 0xb6, 0x89, 0xa6, 0xf1, 0x96, 0x30, 0xf5, 0xd5, 0x1f, 0x35, 0xa6, 0xbe, 0x19, 0x0f, 0x10, 0x67, 0x41, 0x69, 0x33,
0xdb, 0x77, 0x6e, 0x12, 0x29, 0x95, 0xbf, 0x02, 0x18, 0x35, 0x3d, 0x62, 0xc0, 0xcf, 0x44, 0xd3, 0x5e, 0x04, 0xa6, 0xb6, 0xfa, 0x1b, 0xc4, 0xd4, 0x36, 0xeb, 0x19, 0xb1, 0x40, 0x5c, 0xe8, 0x57,
0xed, 0x6f, 0x4c, 0x67, 0x96, 0xaa, 0xfe, 0x04, 0xab, 0x53, 0x3b, 0x0b, 0x31, 0x4a, 0xef, 0xa6, 0xa6, 0x78, 0x72, 0x7f, 0xb2, 0x63, 0xd6, 0x03, 0x63, 0xf8, 0x60, 0x2e, 0xdf, 0xb4, 0xd0, 0x18,
0xde, 0xd4, 0xff, 0xf1, 0xad, 0x72, 0xe5, 0x5d, 0x7f, 0x84, 0xa5, 0x09, 0xf8, 0x35, 0xb3, 0x62, 0x9c, 0x4d, 0x0b, 0xeb, 0xcf, 0x02, 0xd3, 0xc2, 0x19, 0xd3, 0xb6, 0xb3, 0x40, 0x7e, 0x0f, 0x2b,
0x56, 0x57, 0x30, 0xb3, 0x62, 0x26, 0x7e, 0x63, 0xd5, 0x7e, 0x03, 0xdd, 0x0a, 0x30, 0x99, 0x95, 0xb5, 0xe1, 0x95, 0x38, 0x86, 0x15, 0x73, 0xa6, 0xee, 0xe1, 0xc3, 0x2b, 0x65, 0x4a, 0xfd, 0x5f,
0x31, 0x0d, 0xfa, 0xcc, 0xca, 0x98, 0x8a, 0x68, 0xce, 0xdc, 0x8e, 0xf5, 0xc4, 0x7a, 0xb6, 0x05, 0x42, 0xcf, 0x9c, 0x19, 0x89, 0x61, 0xd0, 0x8c, 0xb1, 0x78, 0x78, 0x7f, 0x1e, 0xdb, 0x54, 0x68,
0x8b, 0x42, 0xc3, 0xda, 0xa9, 0xd8, 0xd5, 0xdd, 0xe8, 0x19, 0xe0, 0x5b, 0x8f, 0x78, 0x22, 0x93, 0x8e, 0x2d, 0xa6, 0xc2, 0x19, 0x83, 0x9b, 0xa9, 0x70, 0xd6, 0xb4, 0xe3, 0x2c, 0x90, 0xdf, 0xc1,
0x93, 0x26, 0xfe, 0xab, 0xfe, 0xd9, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0xf8, 0xd4, 0xa1, 0xe3, 0xf2, 0xf4, 0xf8, 0x40, 0x3e, 0x9c, 0x76, 0x5b, 0x6d, 0x2a, 0x19, 0x3a, 0x57, 0x89, 0x94, 0xca,
0xba, 0x16, 0x00, 0x00, 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,
} }

View file

@ -15,6 +15,9 @@ service SeaweedMessaging {
rpc Publish (stream PublishRequest) returns (stream PublishResponse) { rpc Publish (stream PublishRequest) returns (stream PublishResponse) {
} }
rpc DeleteTopic (DeleteTopicRequest) returns (DeleteTopicResponse) {
}
rpc ConfigureTopic (ConfigureTopicRequest) returns (ConfigureTopicResponse) { rpc ConfigureTopic (ConfigureTopicRequest) returns (ConfigureTopicResponse) {
} }
@ -47,6 +50,7 @@ message SubscriberMessage {
int64 message_id = 1; int64 message_id = 1;
} }
AckMessage ack = 2; AckMessage ack = 2;
bool is_close = 3;
} }
message Message { message Message {
@ -54,11 +58,11 @@ message Message {
bytes key = 2; // Message key bytes key = 2; // Message key
bytes value = 3; // Message payload bytes value = 3; // Message payload
map<string, bytes> headers = 4; // Message headers map<string, bytes> headers = 4; // Message headers
bool is_close = 5;
} }
message BrokerMessage { message BrokerMessage {
Message data = 1; Message data = 1;
bool is_close = 2;
} }
message PublishRequest { message PublishRequest {
@ -80,6 +84,14 @@ message PublishResponse {
string new_broker = 1; string new_broker = 1;
} }
RedirectMessage redirect = 2; RedirectMessage redirect = 2;
bool is_closed = 3;
}
message DeleteTopicRequest {
string namespace = 1;
string topic = 2;
}
message DeleteTopicResponse {
} }
message ConfigureTopicRequest { message ConfigureTopicRequest {

View file

@ -14,6 +14,8 @@ It has these top-level messages:
BrokerMessage BrokerMessage
PublishRequest PublishRequest
PublishResponse PublishResponse
DeleteTopicRequest
DeleteTopicResponse
ConfigureTopicRequest ConfigureTopicRequest
ConfigureTopicResponse ConfigureTopicResponse
GetTopicConfigurationRequest GetTopicConfigurationRequest
@ -93,12 +95,13 @@ func (x TopicConfiguration_Partitioning) String() string {
return proto.EnumName(TopicConfiguration_Partitioning_name, int32(x)) return proto.EnumName(TopicConfiguration_Partitioning_name, int32(x))
} }
func (TopicConfiguration_Partitioning) EnumDescriptor() ([]byte, []int) { func (TopicConfiguration_Partitioning) EnumDescriptor() ([]byte, []int) {
return fileDescriptor0, []int{11, 0} return fileDescriptor0, []int{13, 0}
} }
type SubscriberMessage struct { type SubscriberMessage struct {
Init *SubscriberMessage_InitMessage `protobuf:"bytes,1,opt,name=init" json:"init,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"` 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{} } func (m *SubscriberMessage) Reset() { *m = SubscriberMessage{} }
@ -120,6 +123,13 @@ func (m *SubscriberMessage) GetAck() *SubscriberMessage_AckMessage {
return nil return nil
} }
func (m *SubscriberMessage) GetIsClose() bool {
if m != nil {
return m.IsClose
}
return false
}
type SubscriberMessage_InitMessage struct { type SubscriberMessage_InitMessage struct {
Namespace string `protobuf:"bytes,1,opt,name=namespace" json:"namespace,omitempty"` Namespace string `protobuf:"bytes,1,opt,name=namespace" json:"namespace,omitempty"`
Topic string `protobuf:"bytes,2,opt,name=topic" json:"topic,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"` Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"`
Value []byte `protobuf:"bytes,3,opt,name=value,proto3" json:"value,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"` 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{} } func (m *Message) Reset() { *m = Message{} }
@ -234,9 +245,15 @@ func (m *Message) GetHeaders() map[string][]byte {
return nil return nil
} }
func (m *Message) GetIsClose() bool {
if m != nil {
return m.IsClose
}
return false
}
type BrokerMessage struct { type BrokerMessage struct {
Data *Message `protobuf:"bytes,1,opt,name=data" json:"data,omitempty"` 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"`
} }
func (m *BrokerMessage) Reset() { *m = BrokerMessage{} } func (m *BrokerMessage) Reset() { *m = BrokerMessage{} }
@ -251,13 +268,6 @@ func (m *BrokerMessage) GetData() *Message {
return nil return nil
} }
func (m *BrokerMessage) GetIsClose() bool {
if m != nil {
return m.IsClose
}
return false
}
type PublishRequest struct { type PublishRequest struct {
Init *PublishRequest_InitMessage `protobuf:"bytes,1,opt,name=init" json:"init,omitempty"` Init *PublishRequest_InitMessage `protobuf:"bytes,1,opt,name=init" json:"init,omitempty"`
Data *Message `protobuf:"bytes,2,opt,name=data" json:"data,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 { type PublishResponse struct {
Config *PublishResponse_ConfigMessage `protobuf:"bytes,1,opt,name=config" json:"config,omitempty"` Config *PublishResponse_ConfigMessage `protobuf:"bytes,1,opt,name=config" json:"config,omitempty"`
Redirect *PublishResponse_RedirectMessage `protobuf:"bytes,2,opt,name=redirect" json:"redirect,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{} } func (m *PublishResponse) Reset() { *m = PublishResponse{} }
@ -338,6 +349,13 @@ func (m *PublishResponse) GetRedirect() *PublishResponse_RedirectMessage {
return nil return nil
} }
func (m *PublishResponse) GetIsClosed() bool {
if m != nil {
return m.IsClosed
}
return false
}
type PublishResponse_ConfigMessage struct { type PublishResponse_ConfigMessage struct {
PartitionCount int32 `protobuf:"varint,1,opt,name=partition_count,json=partitionCount" json:"partition_count,omitempty"` 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 "" 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 { type ConfigureTopicRequest struct {
Namespace string `protobuf:"bytes,1,opt,name=namespace" json:"namespace,omitempty"` Namespace string `protobuf:"bytes,1,opt,name=namespace" json:"namespace,omitempty"`
Topic string `protobuf:"bytes,2,opt,name=topic" json:"topic,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) Reset() { *m = ConfigureTopicRequest{} }
func (m *ConfigureTopicRequest) String() string { return proto.CompactTextString(m) } func (m *ConfigureTopicRequest) String() string { return proto.CompactTextString(m) }
func (*ConfigureTopicRequest) ProtoMessage() {} 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 { func (m *ConfigureTopicRequest) GetNamespace() string {
if m != nil { if m != nil {
@ -412,7 +462,7 @@ type ConfigureTopicResponse struct {
func (m *ConfigureTopicResponse) Reset() { *m = ConfigureTopicResponse{} } func (m *ConfigureTopicResponse) Reset() { *m = ConfigureTopicResponse{} }
func (m *ConfigureTopicResponse) String() string { return proto.CompactTextString(m) } func (m *ConfigureTopicResponse) String() string { return proto.CompactTextString(m) }
func (*ConfigureTopicResponse) ProtoMessage() {} func (*ConfigureTopicResponse) ProtoMessage() {}
func (*ConfigureTopicResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } func (*ConfigureTopicResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} }
type GetTopicConfigurationRequest struct { type GetTopicConfigurationRequest struct {
Namespace string `protobuf:"bytes,1,opt,name=namespace" json:"namespace,omitempty"` 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) Reset() { *m = GetTopicConfigurationRequest{} }
func (m *GetTopicConfigurationRequest) String() string { return proto.CompactTextString(m) } func (m *GetTopicConfigurationRequest) String() string { return proto.CompactTextString(m) }
func (*GetTopicConfigurationRequest) ProtoMessage() {} 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 { func (m *GetTopicConfigurationRequest) GetNamespace() string {
if m != nil { if m != nil {
@ -445,7 +495,7 @@ type GetTopicConfigurationResponse struct {
func (m *GetTopicConfigurationResponse) Reset() { *m = GetTopicConfigurationResponse{} } func (m *GetTopicConfigurationResponse) Reset() { *m = GetTopicConfigurationResponse{} }
func (m *GetTopicConfigurationResponse) String() string { return proto.CompactTextString(m) } func (m *GetTopicConfigurationResponse) String() string { return proto.CompactTextString(m) }
func (*GetTopicConfigurationResponse) ProtoMessage() {} 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 { func (m *GetTopicConfigurationResponse) GetConfiguration() *TopicConfiguration {
if m != nil { if m != nil {
@ -463,7 +513,7 @@ type FindBrokerRequest struct {
func (m *FindBrokerRequest) Reset() { *m = FindBrokerRequest{} } func (m *FindBrokerRequest) Reset() { *m = FindBrokerRequest{} }
func (m *FindBrokerRequest) String() string { return proto.CompactTextString(m) } func (m *FindBrokerRequest) String() string { return proto.CompactTextString(m) }
func (*FindBrokerRequest) ProtoMessage() {} 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 { func (m *FindBrokerRequest) GetNamespace() string {
if m != nil { if m != nil {
@ -493,7 +543,7 @@ type FindBrokerResponse struct {
func (m *FindBrokerResponse) Reset() { *m = FindBrokerResponse{} } func (m *FindBrokerResponse) Reset() { *m = FindBrokerResponse{} }
func (m *FindBrokerResponse) String() string { return proto.CompactTextString(m) } func (m *FindBrokerResponse) String() string { return proto.CompactTextString(m) }
func (*FindBrokerResponse) ProtoMessage() {} 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 { func (m *FindBrokerResponse) GetBroker() string {
if m != nil { if m != nil {
@ -513,7 +563,7 @@ type TopicConfiguration struct {
func (m *TopicConfiguration) Reset() { *m = TopicConfiguration{} } func (m *TopicConfiguration) Reset() { *m = TopicConfiguration{} }
func (m *TopicConfiguration) String() string { return proto.CompactTextString(m) } func (m *TopicConfiguration) String() string { return proto.CompactTextString(m) }
func (*TopicConfiguration) ProtoMessage() {} 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 { func (m *TopicConfiguration) GetPartitionCount() int32 {
if m != nil { if m != nil {
@ -561,6 +611,8 @@ func init() {
proto.RegisterType((*PublishResponse)(nil), "messaging_pb.PublishResponse") proto.RegisterType((*PublishResponse)(nil), "messaging_pb.PublishResponse")
proto.RegisterType((*PublishResponse_ConfigMessage)(nil), "messaging_pb.PublishResponse.ConfigMessage") proto.RegisterType((*PublishResponse_ConfigMessage)(nil), "messaging_pb.PublishResponse.ConfigMessage")
proto.RegisterType((*PublishResponse_RedirectMessage)(nil), "messaging_pb.PublishResponse.RedirectMessage") 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((*ConfigureTopicRequest)(nil), "messaging_pb.ConfigureTopicRequest")
proto.RegisterType((*ConfigureTopicResponse)(nil), "messaging_pb.ConfigureTopicResponse") proto.RegisterType((*ConfigureTopicResponse)(nil), "messaging_pb.ConfigureTopicResponse")
proto.RegisterType((*GetTopicConfigurationRequest)(nil), "messaging_pb.GetTopicConfigurationRequest") proto.RegisterType((*GetTopicConfigurationRequest)(nil), "messaging_pb.GetTopicConfigurationRequest")
@ -585,6 +637,7 @@ const _ = grpc.SupportPackageIsVersion4
type SeaweedMessagingClient interface { type SeaweedMessagingClient interface {
Subscribe(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error) Subscribe(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error)
Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, 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) ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error)
GetTopicConfiguration(ctx context.Context, in *GetTopicConfigurationRequest, opts ...grpc.CallOption) (*GetTopicConfigurationResponse, error) GetTopicConfiguration(ctx context.Context, in *GetTopicConfigurationRequest, opts ...grpc.CallOption) (*GetTopicConfigurationResponse, error)
FindBroker(ctx context.Context, in *FindBrokerRequest, opts ...grpc.CallOption) (*FindBrokerResponse, 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 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) { func (c *seaweedMessagingClient) ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error) {
out := new(ConfigureTopicResponse) out := new(ConfigureTopicResponse)
err := grpc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/ConfigureTopic", in, out, c.cc, opts...) 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 { type SeaweedMessagingServer interface {
Subscribe(SeaweedMessaging_SubscribeServer) error Subscribe(SeaweedMessaging_SubscribeServer) error
Publish(SeaweedMessaging_PublishServer) error Publish(SeaweedMessaging_PublishServer) error
DeleteTopic(context.Context, *DeleteTopicRequest) (*DeleteTopicResponse, error)
ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error) ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error)
GetTopicConfiguration(context.Context, *GetTopicConfigurationRequest) (*GetTopicConfigurationResponse, error) GetTopicConfiguration(context.Context, *GetTopicConfigurationRequest) (*GetTopicConfigurationResponse, error)
FindBroker(context.Context, *FindBrokerRequest) (*FindBrokerResponse, error) FindBroker(context.Context, *FindBrokerRequest) (*FindBrokerResponse, error)
@ -753,6 +816,24 @@ func (x *seaweedMessagingPublishServer) Recv() (*PublishRequest, error) {
return m, nil 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) { func _SeaweedMessaging_ConfigureTopic_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ConfigureTopicRequest) in := new(ConfigureTopicRequest)
if err := dec(in); err != nil { if err := dec(in); err != nil {
@ -811,6 +892,10 @@ var _SeaweedMessaging_serviceDesc = grpc.ServiceDesc{
ServiceName: "messaging_pb.SeaweedMessaging", ServiceName: "messaging_pb.SeaweedMessaging",
HandlerType: (*SeaweedMessagingServer)(nil), HandlerType: (*SeaweedMessagingServer)(nil),
Methods: []grpc.MethodDesc{ Methods: []grpc.MethodDesc{
{
MethodName: "DeleteTopic",
Handler: _SeaweedMessaging_DeleteTopic_Handler,
},
{ {
MethodName: "ConfigureTopic", MethodName: "ConfigureTopic",
Handler: _SeaweedMessaging_ConfigureTopic_Handler, Handler: _SeaweedMessaging_ConfigureTopic_Handler,
@ -844,65 +929,68 @@ var _SeaweedMessaging_serviceDesc = grpc.ServiceDesc{
func init() { proto.RegisterFile("messaging.proto", fileDescriptor0) } func init() { proto.RegisterFile("messaging.proto", fileDescriptor0) }
var fileDescriptor0 = []byte{ var fileDescriptor0 = []byte{
// 952 bytes of a gzipped FileDescriptorProto // 1002 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x56, 0xdd, 0x6e, 0xe3, 0x44, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x56, 0xdd, 0x6e, 0xe3, 0x54,
0x14, 0xae, 0x9d, 0x34, 0x3f, 0x27, 0x3f, 0xcd, 0x1e, 0xd1, 0x55, 0x30, 0x2d, 0x04, 0x2f, 0x82, 0x10, 0xae, 0xdd, 0xfc, 0x8e, 0x93, 0x34, 0x3b, 0xd0, 0x55, 0xf0, 0xb6, 0x90, 0xf5, 0x22, 0x08,
0x40, 0x21, 0xaa, 0xc2, 0x4d, 0x59, 0xad, 0xb4, 0x6a, 0xa3, 0x2e, 0x1b, 0xd1, 0x76, 0xc3, 0x24, 0x14, 0xa2, 0x2a, 0xdc, 0x94, 0x6a, 0xa5, 0x55, 0x1b, 0xba, 0x34, 0xa2, 0xed, 0x86, 0x93, 0xdc,
0x5c, 0x22, 0xcb, 0xb1, 0x67, 0xd3, 0x51, 0x9d, 0xb1, 0xf1, 0x38, 0x5b, 0xf5, 0x9a, 0x6b, 0xae, 0x22, 0xcb, 0xb1, 0xcf, 0xa6, 0x47, 0x75, 0x8e, 0x8d, 0x8f, 0xb3, 0x55, 0x9f, 0x83, 0x7b, 0x1e,
0x78, 0x15, 0x5e, 0x80, 0x67, 0xe0, 0x02, 0x89, 0xa7, 0x41, 0x1e, 0xff, 0xc4, 0x4e, 0xb2, 0xe9, 0x00, 0x89, 0x3b, 0x5e, 0x80, 0xd7, 0xe0, 0x21, 0x78, 0x06, 0xe4, 0xdf, 0xd8, 0x49, 0x36, 0x5d,
0x52, 0xb4, 0x77, 0x9e, 0x33, 0xdf, 0xf9, 0xce, 0x77, 0xce, 0x9c, 0x33, 0x63, 0xd8, 0x9b, 0x53, 0xb6, 0xda, 0xbb, 0x9c, 0xc9, 0x37, 0x33, 0xdf, 0x99, 0xf9, 0x66, 0x8e, 0x61, 0x67, 0x46, 0x85,
0x21, 0xcc, 0x19, 0xe3, 0xb3, 0x9e, 0xe7, 0xbb, 0x81, 0x8b, 0xf5, 0xd4, 0x60, 0x78, 0x53, 0xfd, 0x30, 0xa6, 0x8c, 0x4f, 0xbb, 0xae, 0xe7, 0xf8, 0x0e, 0xd6, 0x52, 0x83, 0xee, 0x4e, 0xb4, 0xdf,
0xd7, 0x22, 0x3c, 0x1a, 0x2f, 0xa6, 0xc2, 0xf2, 0xd9, 0x94, 0xfa, 0x97, 0x72, 0x8b, 0xe2, 0x73, 0x0b, 0xf0, 0x68, 0x34, 0x9f, 0x08, 0xd3, 0x63, 0x13, 0xea, 0x5d, 0x86, 0x7f, 0x51, 0x7c, 0x01,
0x28, 0x32, 0xce, 0x82, 0xb6, 0xd2, 0x51, 0xba, 0xb5, 0xfe, 0x51, 0x2f, 0xeb, 0xd2, 0x5b, 0x83, 0x05, 0xc6, 0x99, 0xdf, 0x92, 0xda, 0x52, 0x47, 0xe9, 0x1d, 0x74, 0xb3, 0x2e, 0xdd, 0x15, 0x78,
0xf7, 0x86, 0x9c, 0x05, 0xf1, 0x37, 0x91, 0x8e, 0xf8, 0x0c, 0x0a, 0xa6, 0x75, 0xd3, 0x56, 0xa5, 0x77, 0xc0, 0x99, 0x1f, 0xff, 0x26, 0xa1, 0x23, 0x3e, 0x87, 0x6d, 0xc3, 0xbc, 0x69, 0xc9, 0xa1,
0xff, 0x57, 0xf7, 0xf9, 0x9f, 0x5a, 0x37, 0x89, 0x7b, 0xe8, 0xa6, 0xfd, 0xa9, 0x42, 0x2d, 0xc3, 0xff, 0xd7, 0xf7, 0xf9, 0x9f, 0x98, 0x37, 0x89, 0x7b, 0xe0, 0x86, 0x9f, 0x40, 0x85, 0x09, 0xdd,
0x89, 0x07, 0x50, 0xe5, 0xe6, 0x9c, 0x0a, 0xcf, 0xb4, 0xa8, 0xd4, 0x54, 0x25, 0x4b, 0x03, 0x7e, 0xb4, 0x1d, 0x41, 0x5b, 0xdb, 0x6d, 0xa9, 0x53, 0x21, 0x65, 0x26, 0xfa, 0xc1, 0x51, 0xfd, 0x5b,
0x00, 0xbb, 0x81, 0xeb, 0x31, 0x4b, 0x46, 0xab, 0x92, 0x68, 0x11, 0xfa, 0x78, 0xa6, 0x1f, 0xb0, 0x06, 0x25, 0x93, 0x0e, 0xf7, 0xa0, 0xca, 0x8d, 0x19, 0x15, 0xae, 0x61, 0xd2, 0x90, 0x6e, 0x95,
0x80, 0xb9, 0xbc, 0x5d, 0xe8, 0x28, 0xdd, 0x5d, 0xb2, 0x34, 0xa0, 0x01, 0x0d, 0x11, 0x98, 0x7e, 0x2c, 0x0c, 0xf8, 0x31, 0x14, 0x7d, 0xc7, 0x65, 0x66, 0x48, 0xa4, 0x4a, 0xa2, 0x43, 0xe0, 0xe3,
0x30, 0x72, 0x45, 0x84, 0x28, 0x76, 0x94, 0x6e, 0xb3, 0xff, 0xdd, 0x7f, 0xc8, 0xb4, 0x37, 0xce, 0x1a, 0x9e, 0xcf, 0x7c, 0xe6, 0xf0, 0x30, 0x7e, 0x91, 0x2c, 0x0c, 0xa8, 0x43, 0x5d, 0xf8, 0x86,
0x12, 0x90, 0x3c, 0x1f, 0x76, 0xa0, 0x16, 0xb0, 0x39, 0x15, 0x81, 0x39, 0xf7, 0xae, 0x44, 0x7b, 0xe7, 0x0f, 0x1d, 0x11, 0x21, 0x0a, 0x6d, 0xa9, 0xd3, 0xe8, 0x7d, 0xff, 0x3f, 0x8a, 0xd0, 0x1d,
0xb7, 0xa3, 0x74, 0x0b, 0x24, 0x6b, 0xc2, 0x27, 0xd0, 0x10, 0x29, 0xbf, 0xc1, 0xec, 0x76, 0x49, 0x65, 0x03, 0x90, 0x7c, 0x3c, 0x6c, 0x83, 0xe2, 0xb3, 0x19, 0x15, 0xbe, 0x31, 0x73, 0xaf, 0x44,
0xca, 0xaf, 0x2f, 0x8d, 0x43, 0x5b, 0x3f, 0x81, 0x46, 0x2e, 0x0c, 0x02, 0x94, 0x2e, 0x4e, 0x27, 0xab, 0xd8, 0x96, 0x3a, 0xdb, 0x24, 0x6b, 0xc2, 0x67, 0x50, 0x17, 0x69, 0x7c, 0x9d, 0x59, 0xad,
0xe7, 0xe3, 0x49, 0x6b, 0x07, 0xeb, 0x50, 0x39, 0x3f, 0x25, 0x17, 0xc3, 0x70, 0xa5, 0x60, 0x03, 0x52, 0x48, 0xbf, 0xb6, 0x30, 0x0e, 0x2c, 0xed, 0x08, 0xea, 0xb9, 0x34, 0x08, 0x50, 0xba, 0x38,
0xaa, 0x93, 0xe1, 0xe5, 0xf9, 0x78, 0x72, 0x7a, 0x39, 0x6a, 0xa9, 0xda, 0x11, 0xc0, 0xb2, 0xac, 0x19, 0x9f, 0x8d, 0xc6, 0xcd, 0x2d, 0xac, 0x41, 0xe5, 0xec, 0x84, 0x5c, 0x0c, 0x82, 0x93, 0x84,
0x78, 0x08, 0x10, 0x65, 0x46, 0xc3, 0x48, 0x8a, 0x54, 0x53, 0x8d, 0x2d, 0x43, 0x5b, 0xff, 0x4b, 0x75, 0xa8, 0x8e, 0x07, 0x97, 0x67, 0xa3, 0xf1, 0xc9, 0xe5, 0xb0, 0x29, 0xab, 0x07, 0x00, 0x8b,
0x81, 0x72, 0x02, 0xfd, 0x1c, 0x1a, 0xf4, 0x0d, 0xe5, 0x81, 0x11, 0x8a, 0x35, 0xb8, 0x88, 0xd0, 0x8a, 0xe3, 0x3e, 0x40, 0x74, 0x33, 0x1a, 0x64, 0x92, 0x42, 0x36, 0xd5, 0xd8, 0x32, 0xb0, 0xb4,
0x67, 0xea, 0xb1, 0x42, 0x6a, 0x72, 0x63, 0xc2, 0xe6, 0xf4, 0x4a, 0x60, 0x0b, 0x0a, 0x37, 0xf4, 0x7f, 0x25, 0x28, 0x27, 0xd0, 0x2f, 0xa0, 0x4e, 0xdf, 0x50, 0xee, 0xeb, 0x01, 0x59, 0x9d, 0x8b,
0x4e, 0x16, 0xbd, 0x4e, 0xc2, 0xcf, 0xf0, 0x20, 0xde, 0x98, 0xce, 0x82, 0xca, 0x72, 0xd7, 0x49, 0x08, 0x7d, 0x2a, 0x1f, 0x4a, 0x44, 0x09, 0xff, 0x18, 0xb3, 0x19, 0xbd, 0x12, 0xd8, 0x84, 0xed,
0xb4, 0xc0, 0x67, 0x50, 0xbe, 0xa6, 0xa6, 0x4d, 0x7d, 0xd1, 0x2e, 0x76, 0x0a, 0xdd, 0x5a, 0x5f, 0x1b, 0x7a, 0x17, 0x16, 0xbd, 0x46, 0x82, 0x9f, 0x41, 0x23, 0xde, 0x18, 0xf6, 0x3c, 0x6a, 0x67,
0xcf, 0x17, 0x39, 0x29, 0xe7, 0xcb, 0x08, 0x74, 0xce, 0x03, 0xff, 0x8e, 0x24, 0x2e, 0xda, 0x53, 0x8d, 0x44, 0x07, 0x7c, 0x0e, 0xe5, 0x6b, 0x6a, 0x58, 0xd4, 0x13, 0xad, 0x42, 0x7b, 0xbb, 0xa3,
0xa8, 0x67, 0x37, 0x92, 0xa8, 0x51, 0x13, 0xe4, 0xa3, 0xaa, 0x99, 0xa8, 0x4f, 0xd5, 0x13, 0x45, 0xf4, 0xb4, 0x7c, 0x91, 0x93, 0x72, 0x9e, 0x47, 0xa0, 0x33, 0xee, 0x7b, 0x77, 0x24, 0x71, 0xc9,
0xff, 0x09, 0x1a, 0x67, 0xbe, 0x7b, 0xb3, 0x6c, 0xeb, 0x2f, 0xa1, 0x68, 0x9b, 0x81, 0x19, 0xb7, 0xa9, 0xa4, 0x98, 0x57, 0xc9, 0x31, 0xd4, 0xb2, 0x3e, 0x09, 0xa1, 0x48, 0x1f, 0x79, 0x42, 0x72,
0xf5, 0xfe, 0x46, 0x1d, 0x44, 0x42, 0xf0, 0x43, 0xa8, 0x30, 0x61, 0x58, 0x8e, 0x2b, 0x22, 0xe2, 0x86, 0xd0, 0xb1, 0x7c, 0x24, 0x69, 0xc7, 0x50, 0x3f, 0xf5, 0x9c, 0x9b, 0xc5, 0x30, 0x7c, 0x05,
0x0a, 0x29, 0x33, 0x31, 0x08, 0x97, 0xfa, 0x3f, 0x0a, 0x34, 0x47, 0x8b, 0xa9, 0xc3, 0xc4, 0x35, 0x05, 0xcb, 0xf0, 0x8d, 0x78, 0x18, 0x76, 0xd7, 0x52, 0x24, 0x21, 0x44, 0xfb, 0x47, 0x82, 0xc6,
0xa1, 0xbf, 0x2c, 0xa8, 0x08, 0xdb, 0x3d, 0x3b, 0x2f, 0xdd, 0x3c, 0x71, 0x1e, 0xbb, 0x61, 0x58, 0x70, 0x3e, 0xb1, 0x99, 0xb8, 0x26, 0xf4, 0xd7, 0x39, 0x15, 0xc1, 0x24, 0x64, 0x47, 0xa9, 0x93,
0x12, 0x59, 0xea, 0xbd, 0xb2, 0x34, 0xe3, 0x3d, 0x0f, 0x86, 0xfe, 0x9b, 0x0a, 0x7b, 0xa9, 0x60, 0xf7, 0xce, 0x63, 0xd7, 0xcc, 0x51, 0x92, 0x5b, 0xbe, 0x37, 0xb7, 0xaa, 0x7f, 0xe0, 0xc1, 0xd0,
0xe1, 0xb9, 0x5c, 0x50, 0x1c, 0x40, 0xc9, 0x72, 0xf9, 0x6b, 0x36, 0xdb, 0x7c, 0x1f, 0xac, 0xc0, 0xfe, 0x90, 0x61, 0x27, 0x25, 0x2c, 0x5c, 0x87, 0x0b, 0x8a, 0x7d, 0x28, 0x99, 0x0e, 0x7f, 0xcd,
0x7b, 0x03, 0x89, 0x4d, 0x74, 0xc7, 0xae, 0x38, 0x84, 0x8a, 0x4f, 0x6d, 0xe6, 0x53, 0x2b, 0x88, 0xa6, 0xeb, 0x57, 0xc5, 0x12, 0xbc, 0xdb, 0x0f, 0xb1, 0x09, 0xef, 0xd8, 0x15, 0x07, 0x50, 0xf1,
0x13, 0xfd, 0x66, 0x3b, 0x0d, 0x89, 0xd1, 0x09, 0x51, 0xea, 0xae, 0x9d, 0x40, 0x23, 0x17, 0x03, 0xa8, 0xc5, 0x3c, 0x6a, 0xfa, 0xf1, 0x45, 0xbf, 0xdd, 0x1c, 0x86, 0xc4, 0xe8, 0x24, 0x50, 0xea,
0xbf, 0x80, 0xbd, 0x34, 0x03, 0xc3, 0x72, 0x17, 0x3c, 0x3a, 0x89, 0x5d, 0xd2, 0x4c, 0xcd, 0x83, 0x8e, 0x4f, 0xa0, 0x9a, 0x68, 0xc2, 0x8a, 0x57, 0x47, 0x25, 0x16, 0x85, 0xa5, 0x1e, 0x41, 0x3d,
0xd0, 0xaa, 0x1d, 0xc3, 0xde, 0x0a, 0x6d, 0x38, 0x19, 0x9c, 0xde, 0x1a, 0x53, 0xd9, 0x28, 0x69, 0x47, 0x00, 0xbf, 0x84, 0x9d, 0xf4, 0x7a, 0xba, 0xe9, 0xcc, 0x79, 0xd4, 0xa6, 0x22, 0x69, 0xa4,
0x0d, 0xe9, 0x6d, 0xd4, 0x39, 0xfa, 0xef, 0x0a, 0xec, 0x47, 0xc1, 0x16, 0x3e, 0x9d, 0x84, 0x05, 0xe6, 0x7e, 0x60, 0x55, 0x0f, 0x61, 0x67, 0x29, 0x67, 0x30, 0x36, 0x9c, 0xde, 0xea, 0x93, 0x50,
0x4c, 0xce, 0xfc, 0x21, 0xb5, 0x7f, 0x01, 0x0d, 0x2b, 0x26, 0x33, 0xd3, 0xfa, 0xd7, 0xfa, 0x9d, 0x2a, 0x69, 0x81, 0xe9, 0x6d, 0xa4, 0x1d, 0xed, 0x1c, 0xf0, 0x07, 0x6a, 0x53, 0x9f, 0x8e, 0x83,
0x7c, 0x25, 0x64, 0x98, 0x41, 0x16, 0x47, 0xf2, 0x6e, 0x7a, 0x1b, 0x1e, 0xaf, 0x8a, 0x8a, 0xaa, 0xca, 0x26, 0x62, 0x78, 0x8f, 0xa6, 0x68, 0xbb, 0xf0, 0x51, 0x2e, 0x52, 0x54, 0x03, 0xed, 0x37,
0xa6, 0x13, 0x38, 0xf8, 0x9e, 0x06, 0x1b, 0x18, 0x1e, 0xae, 0x5a, 0x9f, 0xc1, 0xe1, 0x5b, 0x38, 0x09, 0x76, 0xa3, 0xdb, 0xcc, 0xbd, 0x07, 0x27, 0xc1, 0x97, 0x50, 0x37, 0xe3, 0x60, 0x46, 0xda,
0xe3, 0x06, 0x59, 0x4b, 0x4b, 0x79, 0x58, 0x5a, 0x16, 0x3c, 0x7a, 0xc1, 0xb8, 0x1d, 0x95, 0xfe, 0x7d, 0xa5, 0xd7, 0xce, 0xf7, 0x21, 0x4c, 0xd3, 0xcf, 0xe2, 0x48, 0xde, 0x4d, 0x6b, 0xc1, 0xe3,
0xff, 0xd4, 0x59, 0x83, 0x8a, 0x67, 0xfa, 0xd9, 0x16, 0x4f, 0xd7, 0xfa, 0xd7, 0x80, 0xd9, 0x20, 0x65, 0x52, 0x31, 0x5f, 0x02, 0x7b, 0x3f, 0x52, 0x7f, 0x4d, 0x84, 0x07, 0x94, 0x66, 0x0a, 0xfb,
0x71, 0x0a, 0x8f, 0xa1, 0x94, 0x6b, 0x81, 0x78, 0xa5, 0xff, 0xa1, 0x02, 0xae, 0x0b, 0x7f, 0xe7, 0x6f, 0x89, 0x19, 0xcb, 0x73, 0xe5, 0x5a, 0xd2, 0xfb, 0x5d, 0xcb, 0x84, 0x47, 0x2f, 0x19, 0xb7,
0x8e, 0xc3, 0x8f, 0x01, 0x2c, 0xd7, 0x71, 0xa8, 0x25, 0xb5, 0x44, 0x22, 0x33, 0x96, 0xf0, 0x9d, 0xa2, 0xde, 0x3e, 0xa4, 0xce, 0x2a, 0x54, 0x5c, 0xc3, 0xcb, 0x0e, 0x58, 0x7a, 0xd6, 0xbe, 0x01,
0xf0, 0xa9, 0xe7, 0x30, 0x6b, 0xd9, 0x0f, 0x55, 0x92, 0x35, 0xe1, 0xa7, 0x50, 0x67, 0xc2, 0x08, 0xcc, 0x26, 0x89, 0xaf, 0xf0, 0x18, 0x4a, 0x39, 0x8d, 0xc5, 0x27, 0xed, 0x2f, 0x19, 0x70, 0x95,
0x7c, 0x93, 0x0b, 0x46, 0x79, 0x20, 0x5f, 0xaa, 0x0a, 0xa9, 0x31, 0x31, 0x49, 0x4c, 0xf8, 0x0a, 0xf8, 0x3b, 0x4b, 0x1a, 0x3f, 0x05, 0x30, 0x1d, 0xdb, 0xa6, 0x66, 0xc8, 0x25, 0x22, 0x99, 0xb1,
0x6a, 0x51, 0x58, 0x97, 0x33, 0x3e, 0x93, 0x8f, 0x4d, 0x73, 0x75, 0xbc, 0xd6, 0x93, 0xe8, 0x8d, 0x04, 0xaf, 0x94, 0x47, 0x5d, 0x9b, 0x99, 0x0b, 0x3d, 0x54, 0x49, 0xd6, 0x84, 0x4f, 0xa1, 0xc6,
0x12, 0xa9, 0x8c, 0xcf, 0x48, 0x96, 0x41, 0x7f, 0x0e, 0xf5, 0xec, 0x26, 0x22, 0x34, 0xaf, 0x5c, 0x84, 0xee, 0x7b, 0x06, 0x17, 0x8c, 0x72, 0x3f, 0x7c, 0x27, 0x2b, 0x44, 0x61, 0x62, 0x9c, 0x98,
0x7e, 0xb5, 0x70, 0x9c, 0x1f, 0xe8, 0xdd, 0x4b, 0x53, 0x5c, 0xb7, 0x76, 0xb0, 0x06, 0xe5, 0x64, 0xf0, 0x15, 0x28, 0x51, 0x5a, 0x87, 0x33, 0x3e, 0x0d, 0xb7, 0x74, 0x63, 0x79, 0xb8, 0x57, 0x2f,
0xa1, 0x60, 0x13, 0x80, 0xb8, 0x0b, 0x6e, 0x13, 0x77, 0xca, 0x78, 0x4b, 0xed, 0xff, 0x5d, 0x80, 0xd1, 0x1d, 0x26, 0x54, 0x19, 0x9f, 0x92, 0x6c, 0x04, 0xed, 0x05, 0xd4, 0xb2, 0x7f, 0x22, 0x42,
0xd6, 0x98, 0x9a, 0xb7, 0x94, 0xda, 0x97, 0x89, 0x0a, 0x7c, 0x05, 0xd5, 0xf4, 0x45, 0xc5, 0x4f, 0xe3, 0xca, 0xe1, 0x57, 0x73, 0xdb, 0xfe, 0x89, 0xde, 0x9d, 0x1b, 0xe2, 0xba, 0xb9, 0x85, 0x0a,
0xee, 0x79, 0x6a, 0xb5, 0x8f, 0xf2, 0x80, 0xdc, 0x4d, 0xae, 0xef, 0x74, 0x95, 0x63, 0x05, 0x2f, 0x94, 0x93, 0x83, 0x84, 0x0d, 0x00, 0xe2, 0xcc, 0xb9, 0x45, 0x9c, 0x09, 0xe3, 0x4d, 0xb9, 0xf7,
0xa0, 0x1c, 0xdf, 0x1a, 0x78, 0xb0, 0xed, 0xce, 0xd5, 0x0e, 0xb7, 0x5e, 0x35, 0x31, 0xdb, 0xcf, 0x67, 0x01, 0x9a, 0x23, 0x6a, 0xdc, 0x52, 0x6a, 0x5d, 0x26, 0x2c, 0xf0, 0x15, 0x54, 0xd3, 0xf7,
0xd0, 0xcc, 0x0f, 0x15, 0x3e, 0xc9, 0xbb, 0x6d, 0xbc, 0x07, 0xb4, 0xcf, 0xb6, 0x83, 0x92, 0x10, 0x1c, 0x3f, 0xbb, 0xe7, 0xa1, 0x57, 0x9f, 0xe4, 0x01, 0xb9, 0xc7, 0x42, 0xdb, 0xea, 0x48, 0x87,
0xe8, 0xc3, 0xfe, 0xc6, 0x29, 0xc2, 0x95, 0xdf, 0xa3, 0x6d, 0xe3, 0xab, 0x1d, 0xbd, 0x13, 0x36, 0x12, 0x5e, 0x40, 0x39, 0xde, 0x59, 0xb8, 0xb7, 0x69, 0xe3, 0xab, 0xfb, 0x1b, 0x17, 0x5d, 0x1c,
0x8d, 0xf9, 0x23, 0xc0, 0xb2, 0xd7, 0x57, 0x4b, 0xbe, 0x36, 0x6a, 0x5a, 0xe7, 0xed, 0x80, 0x84, 0x6d, 0x0c, 0x4a, 0x66, 0x03, 0xe0, 0x92, 0x7a, 0x57, 0xd7, 0x8c, 0xfa, 0x74, 0x03, 0x22, 0x89,
0xf2, 0x4c, 0x87, 0x96, 0x88, 0x0e, 0xf6, 0xb5, 0xe8, 0x59, 0x4e, 0xd8, 0x7f, 0x67, 0xcd, 0xf4, 0x8c, 0xbf, 0x40, 0x23, 0x3f, 0xaa, 0xf8, 0x2c, 0xef, 0xb6, 0x76, 0xbb, 0xa8, 0x9f, 0x6f, 0x06,
0x8c, 0x47, 0xe1, 0x2f, 0xe6, 0xb4, 0x24, 0xff, 0x34, 0xbf, 0xfd, 0x37, 0x00, 0x00, 0xff, 0xff, 0xa5, 0xe1, 0x3d, 0xd8, 0x5d, 0x3b, 0x9b, 0xb8, 0xf4, 0x35, 0xb8, 0x69, 0x29, 0xa8, 0x07, 0xef,
0xad, 0x6b, 0x26, 0x8c, 0x7c, 0x0a, 0x00, 0x00, 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,
} }

View file

@ -390,8 +390,12 @@ func (fs *FilerServer) KeepConnected(stream filer_pb.SeaweedFiler_KeepConnectedS
} }
clientName := fmt.Sprintf("%s:%d", req.Name, req.GrpcPort) 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.brokersLock.Lock()
fs.brokers[clientName] = true fs.brokers[clientName] = m
glog.V(0).Infof("+ broker %v", clientName) glog.V(0).Infof("+ broker %v", clientName)
fs.brokersLock.Unlock() 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
}

View file

@ -82,7 +82,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
lastReadTime = time.Unix(0, processedTsNs) 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.listenersLock.Lock()
fs.listenersCond.Wait() fs.listenersCond.Wait()
fs.listenersLock.Unlock() fs.listenersLock.Unlock()

View file

@ -64,7 +64,7 @@ type FilerServer struct {
listenersLock sync.Mutex listenersLock sync.Mutex
listenersCond *sync.Cond listenersCond *sync.Cond
brokers map[string]bool brokers map[string]map[string]bool
brokersLock sync.Mutex brokersLock sync.Mutex
} }
@ -73,7 +73,7 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
fs = &FilerServer{ fs = &FilerServer{
option: option, option: option,
grpcDialOption: security.LoadClientTLS(util.GetViper(), "grpc.filer"), 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) fs.listenersCond = sync.NewCond(&fs.listenersLock)

View file

@ -12,8 +12,9 @@ import (
) )
func (logBuffer *LogBuffer) LoopProcessLogData( func (logBuffer *LogBuffer) LoopProcessLogData(
startTreadTime time.Time, waitForDataFn func() bool, startTreadTime time.Time,
eachLogDataFn func(logEntry *filer_pb.LogEntry) error) (processed int64, err error) { waitForDataFn func() bool,
eachLogDataFn func(logEntry *filer_pb.LogEntry) error) (err error) {
// loop through all messages // loop through all messages
var bytesBuf *bytes.Buffer var bytesBuf *bytes.Buffer
lastReadTime := startTreadTime lastReadTime := startTreadTime
@ -66,7 +67,6 @@ func (logBuffer *LogBuffer) LoopProcessLogData(
pos += 4 + int(size) pos += 4 + int(size)
batchSize++ batchSize++
processed++
} }
// fmt.Printf("sent message ts[%d,%d] size %d\n", startReadTime.UnixNano(), lastReadTime.UnixNano(), batchSize) // fmt.Printf("sent message ts[%d,%d] size %d\n", startReadTime.UnixNano(), lastReadTime.UnixNano(), batchSize)