From 07006dedd6e6e191f1a6e77d9561688bb4b281c0 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Tue, 25 May 2021 17:21:09 -0600 Subject: [PATCH 01/25] [bug] Fix reader latest position (#525) * Fix reader latest position Currently, using reader.latest fails because we discard all messages less than the startPositionID, this causes an issue with the reader as we filter all messages as all messages are less than latest message id * fix lint * Fix Lint --- pulsar/consumer_partition.go | 4 ++++ pulsar/impl_message.go | 15 +++++++++++++++ pulsar/message.go | 5 ++--- 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 031e0a3a6d..94a914db49 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -575,6 +575,10 @@ func (pc *partitionConsumer) messageShouldBeDiscarded(msgID trackingMessageID) b if pc.startMessageID.Undefined() { return false } + // if we start at latest message, we should never discard + if pc.options.startMessageID.equal(latestMessageID) { + return false + } if pc.options.startMessageIDInclusive { return pc.startMessageID.greater(msgID.messageID) diff --git a/pulsar/impl_message.go b/pulsar/impl_message.go index c358c22e66..a84fc2dee6 100644 --- a/pulsar/impl_message.go +++ b/pulsar/impl_message.go @@ -19,6 +19,7 @@ package pulsar import ( "fmt" + "math" "math/big" "strings" "sync" @@ -36,6 +37,20 @@ type messageID struct { partitionIdx int32 } +var latestMessageID = messageID{ + ledgerID: math.MaxInt64, + entryID: math.MaxInt64, + batchIdx: -1, + partitionIdx: -1, +} + +var earliestMessageID = messageID{ + ledgerID: -1, + entryID: -1, + batchIdx: -1, + partitionIdx: -1, +} + type trackingMessageID struct { messageID diff --git a/pulsar/message.go b/pulsar/message.go index 397c51e992..55ecbcc9bc 100644 --- a/pulsar/message.go +++ b/pulsar/message.go @@ -18,7 +18,6 @@ package pulsar import ( - "math" "time" ) @@ -129,10 +128,10 @@ func DeserializeMessageID(data []byte) (MessageID, error) { // EarliestMessageID returns a messageID that points to the earliest message available in a topic func EarliestMessageID() MessageID { - return newMessageID(-1, -1, -1, -1) + return earliestMessageID } // LatestMessage returns a messageID that points to the latest message func LatestMessageID() MessageID { - return newMessageID(math.MaxInt64, math.MaxInt64, -1, -1) + return latestMessageID } From a03349ac0466a6c7cf595914f6d01d4425eb4aeb Mon Sep 17 00:00:00 2001 From: yorkxyzhang <516302150@qq.com> Date: Tue, 1 Jun 2021 15:15:52 +0800 Subject: [PATCH 02/25] add LedgerId,EntryId,BatchIdx,PartitionIdx func for MessageId interface (#529) * [feature] add LedgerId,EntryId,BatchIdx,PartitionIdx func for MessageId interface * [feature] add a new unit test for MessageId Get func * [format] adjust format --- pulsar/impl_message.go | 16 ++++++++++++++++ pulsar/impl_message_test.go | 9 +++++++++ pulsar/message.go | 12 ++++++++++++ pulsar/reader_test.go | 16 ++++++++++++++++ 4 files changed, 53 insertions(+) diff --git a/pulsar/impl_message.go b/pulsar/impl_message.go index a84fc2dee6..c4f215c5e3 100644 --- a/pulsar/impl_message.go +++ b/pulsar/impl_message.go @@ -123,6 +123,22 @@ func (id messageID) Serialize() []byte { return data } +func (id messageID) LedgerID() int64 { + return id.ledgerID +} + +func (id messageID) EntryID() int64 { + return id.entryID +} + +func (id messageID) BatchIdx() int32 { + return id.batchIdx +} + +func (id messageID) PartitionIdx() int32 { + return id.partitionIdx +} + func (id messageID) String() string { return fmt.Sprintf("%d:%d:%d", id.ledgerID, id.entryID, id.partitionIdx) } diff --git a/pulsar/impl_message_test.go b/pulsar/impl_message_test.go index 4e8b64451e..3421ab4a43 100644 --- a/pulsar/impl_message_test.go +++ b/pulsar/impl_message_test.go @@ -45,6 +45,15 @@ func TestMessageId(t *testing.T) { assert.Nil(t, id) } +func TestMessageIdGetFuncs(t *testing.T) { + // test LedgerId,EntryId,BatchIdx,PartitionIdx + id := newMessageID(1, 2, 3, 4) + assert.Equal(t, int64(1), id.LedgerID()) + assert.Equal(t, int64(2), id.EntryID()) + assert.Equal(t, int32(3), id.BatchIdx()) + assert.Equal(t, int32(4), id.PartitionIdx()) +} + func TestAckTracker(t *testing.T) { tracker := newAckTracker(1) assert.Equal(t, true, tracker.ack(0)) diff --git a/pulsar/message.go b/pulsar/message.go index 55ecbcc9bc..2a4343f352 100644 --- a/pulsar/message.go +++ b/pulsar/message.go @@ -119,6 +119,18 @@ type Message interface { type MessageID interface { // Serialize the message id into a sequence of bytes that can be stored somewhere else Serialize() []byte + + // Get the message ledgerID + LedgerID() int64 + + // Get the message entryID + EntryID() int64 + + // Get the message batchIdx + BatchIdx() int32 + + // Get the message partitionIdx + PartitionIdx() int32 } // DeserializeMessageID reconstruct a MessageID object from its serialized representation diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index f72ba1d1a0..618f5ab9af 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -391,6 +391,22 @@ func (id *myMessageID) Serialize() []byte { return id.data } +func (id *myMessageID) LedgerID() int64 { + return id.LedgerID() +} + +func (id *myMessageID) EntryID() int64 { + return id.EntryID() +} + +func (id *myMessageID) BatchIdx() int32 { + return id.BatchIdx() +} + +func (id *myMessageID) PartitionIdx() int32 { + return id.PartitionIdx() +} + func TestReaderOnSpecificMessageWithCustomMessageID(t *testing.T) { client, err := NewClient(ClientOptions{ URL: lookupURL, From 8a78d2cae985455c50f16b7160cfdbf9495f902d Mon Sep 17 00:00:00 2001 From: jony montana Date: Wed, 2 Jun 2021 11:15:21 +0800 Subject: [PATCH 03/25] fix: timeout guarantee for RequestOnCnx (#492) Signed-off-by: jonyhy96 Fixes #491 ### Motivation see #491 ### Modifications add timeout guarantee to RequestOnCnx ### Verifying this change - [x] Make sure that the change passes the CI checks. --- pulsar/internal/rpc_client.go | 44 ++++++++++++++++++++--------------- 1 file changed, 25 insertions(+), 19 deletions(-) diff --git a/pulsar/internal/rpc_client.go b/pulsar/internal/rpc_client.go index b51e4e3d91..1374747ab8 100644 --- a/pulsar/internal/rpc_client.go +++ b/pulsar/internal/rpc_client.go @@ -21,7 +21,6 @@ import ( "errors" "net" "net/url" - "sync" "sync/atomic" "time" @@ -31,6 +30,16 @@ import ( "github.com/gogo/protobuf/proto" ) +var ( + // ErrRequestTimeOut happens when request not finished in given requestTimeout. + ErrRequestTimeOut = errors.New("request timed out") +) + +type result struct { + *RPCResult + error +} + type RPCResult struct { Response *pb.BaseCommand Cnx Connection @@ -121,14 +130,10 @@ func (c *rpcClient) Request(logicalAddr *url.URL, physicalAddr *url.URL, request return nil, err } - type Res struct { - *RPCResult - error - } - ch := make(chan Res, 10) + ch := make(chan result, 1) cnx.SendRequest(requestID, baseCommand(cmdType, message), func(response *pb.BaseCommand, err error) { - ch <- Res{&RPCResult{ + ch <- result{&RPCResult{ Cnx: cnx, Response: response, }, err} @@ -139,29 +144,30 @@ func (c *rpcClient) Request(logicalAddr *url.URL, physicalAddr *url.URL, request case res := <-ch: return res.RPCResult, res.error case <-time.After(c.requestTimeout): - return nil, errors.New("request timed out") + return nil, ErrRequestTimeOut } } func (c *rpcClient) RequestOnCnx(cnx Connection, requestID uint64, cmdType pb.BaseCommand_Type, message proto.Message) (*RPCResult, error) { c.metrics.RPCRequestCount.Inc() - wg := sync.WaitGroup{} - wg.Add(1) - rpcResult := &RPCResult{ - Cnx: cnx, - } + ch := make(chan result, 1) - var rpcErr error cnx.SendRequest(requestID, baseCommand(cmdType, message), func(response *pb.BaseCommand, err error) { - rpcResult.Response = response - rpcErr = err - wg.Done() + ch <- result{&RPCResult{ + Cnx: cnx, + Response: response, + }, err} + close(ch) }) - wg.Wait() - return rpcResult, rpcErr + select { + case res := <-ch: + return res.RPCResult, res.error + case <-time.After(c.requestTimeout): + return nil, ErrRequestTimeOut + } } func (c *rpcClient) RequestOnCnxNoWait(cnx Connection, cmdType pb.BaseCommand_Type, message proto.Message) error { From cb72395fb53fe24f20c75e2123dc9cae38603eaf Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Tue, 15 Jun 2021 09:27:09 +0800 Subject: [PATCH 04/25] fix nil pointer error with `GetPartitionedTopicMetadata` (#536) * fix nil pointer error * update * fix lint * add more context in error msg --- pulsar/internal/lookup_service.go | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/pulsar/internal/lookup_service.go b/pulsar/internal/lookup_service.go index eb630771be..0602080d3e 100644 --- a/pulsar/internal/lookup_service.go +++ b/pulsar/internal/lookup_service.go @@ -190,11 +190,23 @@ func (ls *lookupService) GetPartitionedTopicMetadata(topic string) (*Partitioned } ls.log.Debugf("Got topic{%s} partitioned metadata response: %+v", topic, res) - if res.Response.PartitionMetadataResponse.Error != nil { - return nil, errors.New(res.Response.PartitionMetadataResponse.GetError().String()) + var partitionedTopicMetadata PartitionedTopicMetadata + + if res.Response.Error != nil { + return nil, errors.New(res.Response.GetError().String()) + } + + if res.Response.PartitionMetadataResponse != nil { + if res.Response.PartitionMetadataResponse.Error != nil { + return nil, errors.New(res.Response.PartitionMetadataResponse.GetError().String()) + } + + partitionedTopicMetadata.Partitions = int(res.Response.PartitionMetadataResponse.GetPartitions()) + } else { + return nil, fmt.Errorf("no partitioned metadata for topic{%s} in lookup response", topic) } - return &PartitionedTopicMetadata{Partitions: int(res.Response.PartitionMetadataResponse.GetPartitions())}, nil + return &partitionedTopicMetadata, nil } func (ls *lookupService) Close() {} From 579984eb5bd516afcff5ba06f48ab3d745900d89 Mon Sep 17 00:00:00 2001 From: cckellogg Date: Wed, 16 Jun 2021 23:03:10 -0700 Subject: [PATCH 05/25] Release locks before calling producer consumer response callbacks. (#542) --- pulsar/internal/connection.go | 46 ++++++++++++++++++++++------------- 1 file changed, 29 insertions(+), 17 deletions(-) diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 9cc12382b4..15dfceca81 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -149,7 +149,9 @@ type connection struct { pendingLock sync.Mutex pendingReqs map[uint64]*request - listeners map[uint64]ConnectionListener + + listenersLock sync.RWMutex + listeners map[uint64]ConnectionListener consumerHandlersLock sync.RWMutex consumerHandlers map[uint64]ConsumerHandler @@ -343,7 +345,8 @@ func (c *connection) run() { go c.reader.readFromConnection() go c.runPingCheck() - c.log.Debugf("Connection run start channel %+v, requestLength %d", c, len(c.incomingRequestsCh)) + c.log.Debugf("Connection run starting with request capacity=%d queued=%d", + cap(c.incomingRequestsCh), len(c.incomingRequestsCh)) defer func() { // all the accesses to the pendingReqs should be happened in this run loop thread, @@ -631,9 +634,9 @@ func (c *connection) handleResponseError(serverError *pb.CommandError) { func (c *connection) handleSendReceipt(response *pb.CommandSendReceipt) { producerID := response.GetProducerId() - c.Lock() + c.listenersLock.RLock() producer, ok := c.listeners[producerID] - c.Unlock() + c.listenersLock.RUnlock() if ok { producer.ReceivedSendReceipt(response) @@ -712,9 +715,6 @@ func (c *connection) handleCloseConsumer(closeConsumer *pb.CommandCloseConsumer) consumerID := closeConsumer.GetConsumerId() c.log.Infof("Broker notification of Closed consumer: %d", consumerID) - c.Lock() - defer c.Unlock() - if consumer, ok := c.consumerHandler(consumerID); ok { consumer.ConnectionClosed() c.DeleteConsumeHandler(consumerID) @@ -727,31 +727,36 @@ func (c *connection) handleCloseProducer(closeProducer *pb.CommandCloseProducer) c.log.Infof("Broker notification of Closed producer: %d", closeProducer.GetProducerId()) producerID := closeProducer.GetProducerId() - c.Lock() - defer c.Unlock() - if producer, ok := c.listeners[producerID]; ok { - producer.ConnectionClosed() + c.listenersLock.Lock() + producer, ok := c.listeners[producerID] + if ok { delete(c.listeners, producerID) + } + c.listenersLock.Unlock() + + // did we find a producer? + if ok { + producer.ConnectionClosed() } else { c.log.WithField("producerID", producerID).Warn("Producer with ID not found while closing producer") } } func (c *connection) RegisterListener(id uint64, listener ConnectionListener) { - c.Lock() - defer c.Unlock() + c.listenersLock.Lock() + defer c.listenersLock.Unlock() c.listeners[id] = listener } func (c *connection) UnregisterListener(id uint64) { - c.Lock() - defer c.Unlock() + c.listenersLock.Lock() + defer c.listenersLock.Unlock() delete(c.listeners, id) } -// Triggers the connection close by forcing the socket to close and +// TriggerClose the connection close by forcing the socket to close and // broadcasting the notification on the close channel func (c *connection) TriggerClose() { c.closeOnce.Do(func() { @@ -781,7 +786,14 @@ func (c *connection) Close() { c.pingTicker.Stop() c.pingCheckTicker.Stop() - for _, listener := range c.listeners { + listeners := make(map[uint64]ConnectionListener) + c.listenersLock.RLock() + for id, listener := range c.listeners { + listeners[id] = listener + } + c.listenersLock.RUnlock() + + for _, listener := range listeners { listener.ConnectionClosed() } From 55c15ef087f3d24b22be98105d918dc039c79109 Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Thu, 17 Jun 2021 17:07:38 +0800 Subject: [PATCH 06/25] #540 fix lookup service not implemented GetTopicsOfNamespace (#541) Fixes #540 ### Motivation When using regex topics consumer with HTTP client URL, the client will fail to query namespace topics and throw failed handshake error. This is because the client sends protobuf requests to HTTP service. ### Modifications add GetTopicsOfNamespace to LookupService interface add related tests ### Verifying this change - [x] Make sure that the change passes the CI checks. --- pulsar/client_impl.go | 21 ------ pulsar/client_impl_test.go | 89 +++++++++++++++++++++++++- pulsar/consumer_regex.go | 2 +- pulsar/internal/lookup_service.go | 56 +++++++++++++++- pulsar/internal/namespace_name.go | 26 ++++++++ pulsar/internal/namespace_name_test.go | 30 +++++++++ 6 files changed, 198 insertions(+), 26 deletions(-) create mode 100644 pulsar/internal/namespace_name.go create mode 100644 pulsar/internal/namespace_name_test.go diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index ee879c8a41..52ef5b1ca7 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -22,13 +22,10 @@ import ( "net/url" "time" - "github.com/gogo/protobuf/proto" - "github.com/sirupsen/logrus" "github.com/apache/pulsar-client-go/pulsar/internal" "github.com/apache/pulsar-client-go/pulsar/internal/auth" - pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" "github.com/apache/pulsar-client-go/pulsar/log" ) @@ -204,21 +201,3 @@ func (c *client) Close() { c.cnxPool.Close() c.lookupService.Close() } - -func (c *client) namespaceTopics(namespace string) ([]string, error) { - id := c.rpcClient.NewRequestID() - req := &pb.CommandGetTopicsOfNamespace{ - RequestId: proto.Uint64(id), - Namespace: proto.String(namespace), - Mode: pb.CommandGetTopicsOfNamespace_PERSISTENT.Enum(), - } - res, err := c.rpcClient.RequestToAnyBroker(id, pb.BaseCommand_GET_TOPICS_OF_NAMESPACE, req) - if err != nil { - return nil, err - } - if res.Response.Error != nil { - return []string{}, newError(LookupError, res.Response.GetError().String()) - } - - return res.Response.GetTopicsOfNamespaceResponse.GetTopics(), nil -} diff --git a/pulsar/client_impl_test.go b/pulsar/client_impl_test.go index 3fb72480d6..b1d128aaf4 100644 --- a/pulsar/client_impl_test.go +++ b/pulsar/client_impl_test.go @@ -28,6 +28,8 @@ import ( "testing" "time" + "github.com/apache/pulsar-client-go/pulsar/internal" + "github.com/apache/pulsar-client-go/pulsar/internal/auth" "github.com/stretchr/testify/assert" ) @@ -381,11 +383,29 @@ func TestNamespaceTopicsNamespaceDoesNotExit(t *testing.T) { // fetch from namespace that does not exist name := generateRandomName() - topics, err := ci.namespaceTopics(fmt.Sprintf("%s/%s", name, name)) + topics, err := ci.lookupService.GetTopicsOfNamespace(fmt.Sprintf("%s/%s", name, name), internal.Persistent) assert.Nil(t, err) assert.Equal(t, 0, len(topics)) } +func TestNamespaceTopicsNamespaceDoesNotExitWebURL(t *testing.T) { + c, err := NewClient(ClientOptions{ + URL: webServiceURL, + }) + if err != nil { + t.Errorf("failed to create client error: %+v", err) + return + } + defer c.Close() + ci := c.(*client) + + // fetch from namespace that does not exist + name := generateRandomName() + topics, err := ci.lookupService.GetTopicsOfNamespace(fmt.Sprintf("%s/%s", name, name), internal.Persistent) + assert.NotNil(t, err) + assert.Equal(t, 0, len(topics)) +} + func TestNamespaceTopics(t *testing.T) { name := generateRandomName() namespace := fmt.Sprintf("public/%s", name) @@ -421,7 +441,70 @@ func TestNamespaceTopics(t *testing.T) { defer c.Close() ci := c.(*client) - topics, err := ci.namespaceTopics(namespace) + topics, err := ci.lookupService.GetTopicsOfNamespace(namespace, internal.Persistent) + if err != nil { + t.Fatal(err) + } + assert.Equal(t, 2, len(topics)) + + // add a non-persistent topic + topicName := fmt.Sprintf("non-persistent://%s/testNonPersistentTopic", namespace) + client, err := NewClient(ClientOptions{ + URL: serviceURL, + }) + assert.Nil(t, err) + defer client.Close() + + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topicName, + }) + + assert.Nil(t, err) + defer producer.Close() + + topics, err = ci.lookupService.GetTopicsOfNamespace(namespace, internal.Persistent) + if err != nil { + t.Fatal(err) + } + assert.Equal(t, 2, len(topics)) +} + +func TestNamespaceTopicsWebURL(t *testing.T) { + name := generateRandomName() + namespace := fmt.Sprintf("public/%s", name) + namespaceURL := fmt.Sprintf("admin/v2/namespaces/%s", namespace) + err := httpPut(namespaceURL, anonymousNamespacePolicy()) + if err != nil { + t.Fatal() + } + defer func() { + _ = httpDelete(fmt.Sprintf("admin/v2/namespaces/%s", namespace)) + }() + + // create topics + topic1 := fmt.Sprintf("%s/topic-1", namespace) + if err := httpPut("admin/v2/persistent/"+topic1, nil); err != nil { + t.Fatal(err) + } + topic2 := fmt.Sprintf("%s/topic-2", namespace) + if err := httpPut("admin/v2/persistent/"+topic2, namespace); err != nil { + t.Fatal(err) + } + defer func() { + _ = httpDelete("admin/v2/persistent/"+topic1, "admin/v2/persistent/"+topic2) + }() + + c, err := NewClient(ClientOptions{ + URL: webServiceURL, + }) + if err != nil { + t.Errorf("failed to create client error: %+v", err) + return + } + defer c.Close() + ci := c.(*client) + + topics, err := ci.lookupService.GetTopicsOfNamespace(namespace, internal.Persistent) if err != nil { t.Fatal(err) } @@ -442,7 +525,7 @@ func TestNamespaceTopics(t *testing.T) { assert.Nil(t, err) defer producer.Close() - topics, err = ci.namespaceTopics(namespace) + topics, err = ci.lookupService.GetTopicsOfNamespace(namespace, internal.Persistent) if err != nil { t.Fatal(err) } diff --git a/pulsar/consumer_regex.go b/pulsar/consumer_regex.go index 7072370652..9e0c125e71 100644 --- a/pulsar/consumer_regex.go +++ b/pulsar/consumer_regex.go @@ -343,7 +343,7 @@ func (c *regexConsumer) unsubscribe(topics []string) { } func (c *regexConsumer) topics() ([]string, error) { - topics, err := c.client.namespaceTopics(c.namespace) + topics, err := c.client.lookupService.GetTopicsOfNamespace(c.namespace, internal.Persistent) if err != nil { return nil, err } diff --git a/pulsar/internal/lookup_service.go b/pulsar/internal/lookup_service.go index 0602080d3e..867644f90e 100644 --- a/pulsar/internal/lookup_service.go +++ b/pulsar/internal/lookup_service.go @@ -34,6 +34,15 @@ type LookupResult struct { PhysicalAddr *url.URL } +// GetTopicsOfNamespaceMode for CommandGetTopicsOfNamespace_Mode +type GetTopicsOfNamespaceMode string + +const ( + Persistent GetTopicsOfNamespaceMode = "PERSISTENT" + NonPersistent = "NON_PERSISTENT" + All = "ALL" +) + // PartitionedTopicMetadata encapsulates a struct for metadata of a partitioned topic type PartitionedTopicMetadata struct { Partitions int `json:"partitions"` // Number of partitions for the topic @@ -49,7 +58,10 @@ type LookupService interface { // the given topic, returns the CommandPartitionedTopicMetadataResponse as the result. GetPartitionedTopicMetadata(topic string) (*PartitionedTopicMetadata, error) - // Allow Lookup Service's internal client to be able to closed + // GetTopicsOfNamespace returns all the topics name for a given namespace. + GetTopicsOfNamespace(namespace string, mode GetTopicsOfNamespaceMode) ([]string, error) + + // Closable Allow Lookup Service's internal client to be able to closed Closable } @@ -209,12 +221,33 @@ func (ls *lookupService) GetPartitionedTopicMetadata(topic string) (*Partitioned return &partitionedTopicMetadata, nil } +func (ls *lookupService) GetTopicsOfNamespace(namespace string, mode GetTopicsOfNamespaceMode) ([]string, error) { + id := ls.rpcClient.NewRequestID() + pbMode := pb.CommandGetTopicsOfNamespace_Mode(pb.CommandGetTopicsOfNamespace_Mode_value[string(mode)]) + req := &pb.CommandGetTopicsOfNamespace{ + RequestId: proto.Uint64(id), + Namespace: proto.String(namespace), + Mode: &pbMode, + } + res, err := ls.rpcClient.RequestToAnyBroker(id, pb.BaseCommand_GET_TOPICS_OF_NAMESPACE, req) + if err != nil { + return nil, err + } + if res.Response.Error != nil { + return []string{}, errors.New(res.Response.GetError().String()) + } + + return res.Response.GetTopicsOfNamespaceResponse.GetTopics(), nil +} + func (ls *lookupService) Close() {} const HTTPLookupServiceBasePathV1 string = "/lookup/v2/destination/" const HTTPLookupServiceBasePathV2 string = "/lookup/v2/topic/" const HTTPAdminServiceV1Format string = "/admin/%s/partitions" const HTTPAdminServiceV2Format string = "/admin/v2/%s/partitions" +const HTTPTopicUnderNamespaceV1 string = "/admin/namespaces/%s/destinations?mode=%s" +const HTTPTopicUnderNamespaceV2 string = "/admin/v2/namespaces/%s/topics?mode=%s" type httpLookupData struct { BrokerURL string `json:"brokerUrl"` @@ -304,6 +337,27 @@ func (h *httpLookupService) GetPartitionedTopicMetadata(topic string) (*Partitio return tMetadata, nil } +func (h *httpLookupService) GetTopicsOfNamespace(namespace string, mode GetTopicsOfNamespaceMode) ([]string, error) { + + format := HTTPTopicUnderNamespaceV2 + if !IsV2Namespace(namespace) { + format = HTTPTopicUnderNamespaceV1 + } + + path := fmt.Sprintf(format, namespace, string(mode)) + + topics := []string{} + + err := h.httpClient.Get(path, &topics) + if err != nil { + return nil, err + } + + h.log.Debugf("Got namespace{%s} mode{%s} topics response: %+v", namespace, mode, topics) + + return topics, nil +} + func (h *httpLookupService) Close() { h.httpClient.Close() } diff --git a/pulsar/internal/namespace_name.go b/pulsar/internal/namespace_name.go new file mode 100644 index 0000000000..5e9fee220a --- /dev/null +++ b/pulsar/internal/namespace_name.go @@ -0,0 +1,26 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package internal + +import "strings" + +func IsV2Namespace(namespace string) bool { + parts := strings.Split(namespace, "/") + // Legacy namespace name that includes cluster name + return len(parts) == 2 +} diff --git a/pulsar/internal/namespace_name_test.go b/pulsar/internal/namespace_name_test.go new file mode 100644 index 0000000000..d365cc7735 --- /dev/null +++ b/pulsar/internal/namespace_name_test.go @@ -0,0 +1,30 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package internal + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestIsV2Namespace(t *testing.T) { + assert.True(t, IsV2Namespace("tenant/default")) + assert.False(t, IsV2Namespace("tenant/cluster/default")) + assert.False(t, IsV2Namespace("default")) +} From d7b4299758f3dacc44f20908c9a5270abbac5248 Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Tue, 22 Jun 2021 20:03:14 +0800 Subject: [PATCH 07/25] regenerate the certs to work with Pulsar 2.8.0 and Java 11 (#548) * regenerate the certs & tokens * revert token changes * fix legacy Common Name field * update conf * fix topic not found * fix auto create topic * always recreate docker image * create topic first * fix ci * print standalone.conf * Revert "create topic first" This reverts commit 704c968bc6caa8724a97653286d011184347ff81. * more log * fix http request with params * cleanup debug log --- .github/workflows/go.yml | 4 + docker-ci.sh | 2 + integration-tests/certs/broker-cert.pem | 112 +++++----------- integration-tests/certs/broker-key.pem | 55 ++++---- integration-tests/certs/cacert.pem | 114 +++++----------- integration-tests/certs/client-cert.pem | 108 ++++----------- integration-tests/certs/client-key.pem | 55 ++++---- integration-tests/certs/generate.sh | 90 +++++++++++-- integration-tests/certs/openssl.cnf | 171 +++++++++++++++++++++--- integration-tests/standalone.conf | 21 ++- pulsar-test-service-start.sh | 1 + pulsar/internal/http_client.go | 8 +- pulsar/internal/lookup_service.go | 6 +- pulsar/internal/lookup_service_test.go | 2 +- 14 files changed, 414 insertions(+), 335 deletions(-) diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml index f1f8658657..c04a463de7 100644 --- a/.github/workflows/go.yml +++ b/.github/workflows/go.yml @@ -8,6 +8,10 @@ jobs: matrix: go-version: [1.11, 1.12, 1.13, 1.14] steps: + - name: clean docker cache + run: | + docker rmi $(docker images -q) -f + df -h - name: Set up Go uses: actions/setup-go@v1 with: diff --git a/docker-ci.sh b/docker-ci.sh index 9694c8b2ea..74a57c125c 100755 --- a/docker-ci.sh +++ b/docker-ci.sh @@ -26,6 +26,8 @@ cd ${SRC_DIR} IMAGE_NAME=pulsar-client-go-test:latest GO_VERSION=${1:-1.12} +docker rmi --force ${IMAGE_NAME} || true +docker rmi --force apachepulsar/pulsar:latest || true docker build -t ${IMAGE_NAME} --build-arg GO_VERSION="golang:${GO_VERSION}" . docker run -i -v ${PWD}:/pulsar-client-go ${IMAGE_NAME} \ diff --git a/integration-tests/certs/broker-cert.pem b/integration-tests/certs/broker-cert.pem index 504534e81c..1725df2ae7 100644 --- a/integration-tests/certs/broker-cert.pem +++ b/integration-tests/certs/broker-cert.pem @@ -1,81 +1,35 @@ -Certificate: - Data: - Version: 3 (0x2) - Serial Number: 9806047019225563732 (0x881613cf2fb7c254) - Signature Algorithm: sha1WithRSAEncryption - Issuer: C=US, ST=CA, O=Apache, OU=Apache Pulsar, CN=localhost - Validity - Not Before: Apr 13 08:01:48 2021 GMT - Not After : Apr 11 08:01:48 2031 GMT - Subject: C=US, ST=CA, O=Apache, OU=Apache Pulsar, CN=localhost - Subject Public Key Info: - Public Key Algorithm: rsaEncryption - Public-Key: (2048 bit) - Modulus: - 00:a2:78:b9:3f:af:e2:d3:15:e4:9a:d1:54:27:93: - a2:6a:f1:db:e0:40:34:d5:72:96:b9:61:c7:c3:02: - 22:5c:03:9d:d9:f6:5c:37:d6:9e:93:e4:64:37:69: - 1b:56:e1:a8:58:09:17:3d:87:63:07:c3:c2:b5:33: - 55:e6:b6:cf:c3:d9:5b:7a:8a:90:5e:19:35:2b:1f: - c1:38:ba:02:7f:49:4e:7f:49:9b:d3:e3:a3:a0:f1: - 55:75:83:e5:75:ec:fb:f8:88:a9:b1:10:2b:14:00: - f8:f9:6c:13:be:9b:ca:64:b1:66:f4:ab:6e:b4:ca: - 10:85:15:84:6c:f1:1a:29:06:15:e3:2e:3e:d5:83: - c5:8d:e9:b7:42:5c:16:e8:72:37:e8:bc:f0:57:59: - 37:3b:14:ad:3d:12:0b:41:12:1b:54:1e:41:de:5f: - 44:58:6b:25:e6:b9:6f:02:bd:f7:ed:43:44:2f:62: - 66:9c:05:1e:99:f9:56:1e:c7:de:09:96:55:ea:dd: - a1:38:42:ae:d7:a1:88:b3:9e:21:ea:a1:7c:6d:76: - 6a:d4:fd:d7:fd:ba:03:7a:90:46:66:29:4f:40:b6: - 95:8b:c3:48:44:c5:00:1b:4c:5e:0c:c2:f5:7a:1f: - 2f:35:38:da:37:b7:05:ce:f9:20:ec:f2:28:1b:8d: - 60:ef - Exponent: 65537 (0x10001) - X509v3 extensions: - X509v3 Basic Constraints: - CA:FALSE - X509v3 Authority Key Identifier: - keyid:59:A0:A4:78:56:E1:56:61:4E:DC:E9:80:FF:EC:66:B6:2F:04:6D:AA - - X509v3 Subject Key Identifier: - 9B:27:81:1A:7A:94:A7:C0:05:1C:6E:23:8D:80:B9:97:0D:46:95:F8 - X509v3 Subject Alternative Name: - DNS:localhost - Signature Algorithm: sha1WithRSAEncryption - 11:a0:ed:8a:27:16:7b:80:f5:12:e5:0a:ed:e4:73:11:55:bf: - 4e:c5:b0:62:c7:0e:74:51:0a:31:34:e3:98:7e:68:32:79:75: - 52:0a:b3:9d:12:f7:47:bd:a4:48:dc:00:d5:46:c5:6f:9c:67: - 59:72:8b:fb:97:95:aa:88:e7:62:d2:3f:16:67:d9:c8:b5:8a: - eb:b2:71:0d:e5:dd:fe:05:bc:8e:79:59:95:d2:e5:d9:16:94: - e7:6f:44:8f:91:22:30:8e:bf:d3:9d:a4:7f:0e:84:77:d3:15: - a0:ed:ed:da:ca:74:99:6d:3e:6a:a3:63:f3:cb:cc:f4:d4:1c: - 63:a7:5c:2c:4c:cb:8b:35:a7:f5:01:1b:df:8b:1c:e5:40:88: - 47:32:f5:f2:f7:9a:35:29:37:0a:7d:77:a0:3d:db:fb:53:95: - af:74:f4:c0:bb:cb:e1:4a:9b:b4:29:32:21:31:50:fd:57:d0: - e1:f2:81:98:8c:06:93:1d:da:c6:89:c2:54:44:5f:85:8b:a9: - e8:23:7b:0a:dc:1e:ab:1a:73:3a:19:f1:89:ad:91:54:1d:c3: - 1a:e9:e1:aa:fa:d9:37:35:19:dd:6f:f1:d2:d8:9e:c2:3e:4c: - 86:26:38:d3:25:5e:03:f7:8e:96:a1:63:d0:a3:3b:a4:5e:ac: - bc:45:90:f2 -----BEGIN CERTIFICATE----- -MIIDlDCCAnygAwIBAgIJAIgWE88vt8JUMA0GCSqGSIb3DQEBBQUAMFcxCzAJBgNV -BAYTAlVTMQswCQYDVQQIDAJDQTEPMA0GA1UECgwGQXBhY2hlMRYwFAYDVQQLDA1B -cGFjaGUgUHVsc2FyMRIwEAYDVQQDDAlsb2NhbGhvc3QwHhcNMjEwNDEzMDgwMTQ4 -WhcNMzEwNDExMDgwMTQ4WjBXMQswCQYDVQQGEwJVUzELMAkGA1UECAwCQ0ExDzAN -BgNVBAoMBkFwYWNoZTEWMBQGA1UECwwNQXBhY2hlIFB1bHNhcjESMBAGA1UEAwwJ -bG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAoni5P6/i -0xXkmtFUJ5OiavHb4EA01XKWuWHHwwIiXAOd2fZcN9aek+RkN2kbVuGoWAkXPYdj -B8PCtTNV5rbPw9lbeoqQXhk1Kx/BOLoCf0lOf0mb0+OjoPFVdYPldez7+IipsRAr -FAD4+WwTvpvKZLFm9KtutMoQhRWEbPEaKQYV4y4+1YPFjem3QlwW6HI36LzwV1k3 -OxStPRILQRIbVB5B3l9EWGsl5rlvAr337UNEL2JmnAUemflWHsfeCZZV6t2hOEKu -16GIs54h6qF8bXZq1P3X/boDepBGZilPQLaVi8NIRMUAG0xeDML1eh8vNTjaN7cF -zvkg7PIoG41g7wIDAQABo2MwYTAJBgNVHRMEAjAAMB8GA1UdIwQYMBaAFFmgpHhW -4VZhTtzpgP/sZrYvBG2qMB0GA1UdDgQWBBSbJ4EaepSnwAUcbiONgLmXDUaV+DAU -BgNVHREEDTALgglsb2NhbGhvc3QwDQYJKoZIhvcNAQEFBQADggEBABGg7YonFnuA -9RLlCu3kcxFVv07FsGLHDnRRCjE045h+aDJ5dVIKs50S90e9pEjcANVGxW+cZ1ly -i/uXlaqI52LSPxZn2ci1iuuycQ3l3f4FvI55WZXS5dkWlOdvRI+RIjCOv9OdpH8O -hHfTFaDt7drKdJltPmqjY/PLzPTUHGOnXCxMy4s1p/UBG9+LHOVAiEcy9fL3mjUp -Nwp9d6A92/tTla909MC7y+FKm7QpMiExUP1X0OHygZiMBpMd2saJwlREX4WLqegj -ewrcHqsaczoZ8YmtkVQdwxrp4ar62Tc1Gd1v8dLYnsI+TIYmONMlXgP3jpahY9Cj -O6RerLxFkPI= +MIIGHzCCBAegAwIBAgICEAAwDQYJKoZIhvcNAQELBQAwgZIxCzAJBgNVBAYTAlVT +MRMwEQYDVQQIDApDYWxpZm9ybmlhMSMwIQYDVQQKDBpBcGFjaGUgU29mdHdhcmUg +Rm91bmRhdGlvbjEPMA0GA1UECwwGUHVsc2FyMRIwEAYDVQQDDAlQdWxzYXIgQ0Ex +JDAiBgkqhkiG9w0BCQEWFWRldkBwdWxzYXIuYXBhY2hlLm9yZzAeFw0yMTA2MjIw +NTU1MzlaFw0zMTA2MjAwNTU1MzlaMIGSMQswCQYDVQQGEwJVUzETMBEGA1UECAwK +Q2FsaWZvcm5pYTEjMCEGA1UECgwaQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24x +DzANBgNVBAsMBlB1bHNhcjESMBAGA1UEAwwJbG9jYWxob3N0MSQwIgYJKoZIhvcN +AQkBFhVkZXZAcHVsc2FyLmFwYWNoZS5vcmcwggEiMA0GCSqGSIb3DQEBAQUAA4IB +DwAwggEKAoIBAQCqtRND1GELHgVfRSNBcvIeZ6o3ARieCKi7UGS13c8GAYJjpHId +9FEOT6K2HhqsdsgAgmNK8LcY5NWMyNascwZN3oW+ftTiXTB5EhOgdOaOnREiVbvQ +cm6RIrQJQGp+Lbv3i3KUZzQ9ziTNu0OturKD2HADfZMMBCySTSZocws+tY3KykGr +MeXsKuF+5WKlxBCJM4qWrC3izhzNLGsocwIkiqTXIaW79Tp0GpAB58j3QgR5rBCI +cbDDtnAoLOuyQTkHakiCKT+kx6DpSbtptgNseeAuZAm8AGIrifSEEiOSPrvLNsem +dkyORK+lupyp4mulXRyyGp8WofnH5qCGsrB/AgMBAAGjggF7MIIBdzAJBgNVHRME +AjAAMBEGCWCGSAGG+EIBAQQEAwIGQDAzBglghkgBhvhCAQ0EJhYkT3BlblNTTCBH +ZW5lcmF0ZWQgU2VydmVyIENlcnRpZmljYXRlMB0GA1UdDgQWBBR14LSJhdIK8UBr +DWvSbfL2PmUK4DCBxwYDVR0jBIG/MIG8gBScEpVAELMJboc1hfTtqTV5sr3OCKGB +mKSBlTCBkjELMAkGA1UEBhMCVVMxEzARBgNVBAgMCkNhbGlmb3JuaWExIzAhBgNV +BAoMGkFwYWNoZSBTb2Z0d2FyZSBGb3VuZGF0aW9uMQ8wDQYDVQQLDAZQdWxzYXIx +EjAQBgNVBAMMCVB1bHNhciBDQTEkMCIGCSqGSIb3DQEJARYVZGV2QHB1bHNhci5h +cGFjaGUub3JnggkAmcALIaQWyzkwDgYDVR0PAQH/BAQDAgWgMBMGA1UdJQQMMAoG +CCsGAQUFBwMBMBQGA1UdEQQNMAuCCWxvY2FsaG9zdDANBgkqhkiG9w0BAQsFAAOC +AgEAUbKo4czkbnjtcaSU0lClp3QdULOk7GHXPqwkGmRBxE//Z2xNgtkDa7k31XNv +9sMH1Y+GzL77JxKDMOYnLLlzOZisGfu7fdJUCb8+UR1JZjr382/V05+ngnlS4VQM +ScIM4o7Fjxs6VLQHTo67IcfMY2XVa3xiSlYnZk7EqGxFM5rTlzNeD8clVBNSDMrZ +0JtHzremhTCA9Uy4e9fEJg2vr5j8IrDz2Gi4mkVOn+4oi6MeS2N8lRB0J26ONra7 +DdcK3b9zPf+1hlzxsDV/uge5qfD7dQQg1160yK8AsqQF2xZC1mGDPfqOJ3uswHt0 +by+Pg9f0FBMzZC/u033jE7+4kLuEqT/LBeW324f9euOZ2fz1GQUos3APZnvddz+G +K3Q92lv7leepDlZpSSXIOp21cRNo3glymMRCa9VplRsUev4TqQYVUyqSzJF2BLVA +5j0mgJzFS2y5FlvgXsyj/QuO8wrAfFmD+7xjZzGNFhQjSCEMMGa/7zcassC3GWfs +4UVtySyGpIfpsfYHP/HPPM7LkbMlAklx0sKFUOUS82EV0AXHyC6wbR0f16TkPdH0 +89RnP6NrWeOL2un5GPQB+yH7///oXkhMk6M2y/cHvGCBMC5FW/+zRkmdyEzJZWYm +eATWqjETDTkgBu84NAO0YZ2FtHz2gc/xDeQeDY97Q8maIxU= -----END CERTIFICATE----- diff --git a/integration-tests/certs/broker-key.pem b/integration-tests/certs/broker-key.pem index ef7049e74d..bc87c60d8c 100644 --- a/integration-tests/certs/broker-key.pem +++ b/integration-tests/certs/broker-key.pem @@ -1,27 +1,28 @@ ------BEGIN RSA PRIVATE KEY----- -MIIEogIBAAKCAQEAoni5P6/i0xXkmtFUJ5OiavHb4EA01XKWuWHHwwIiXAOd2fZc -N9aek+RkN2kbVuGoWAkXPYdjB8PCtTNV5rbPw9lbeoqQXhk1Kx/BOLoCf0lOf0mb -0+OjoPFVdYPldez7+IipsRArFAD4+WwTvpvKZLFm9KtutMoQhRWEbPEaKQYV4y4+ -1YPFjem3QlwW6HI36LzwV1k3OxStPRILQRIbVB5B3l9EWGsl5rlvAr337UNEL2Jm -nAUemflWHsfeCZZV6t2hOEKu16GIs54h6qF8bXZq1P3X/boDepBGZilPQLaVi8NI -RMUAG0xeDML1eh8vNTjaN7cFzvkg7PIoG41g7wIDAQABAoIBAGfai2+uWmtbvzlH -PWfe8x8xYr9By4P1L2tomWyJ3LS8ixZcaC45PjyC7ZyNzzAGJrm/dd+avy6jb76s -BejoSn6CXftCv22m51mdfsFoBVif31F2F+mlxV6HZX0sxmxPA/GmSfIox6CNzpiB -AU4B/6nFzG3xd16RG2Y2juOkanh1qn8fY+pDspV1LG1v4MBRrFfPPRdJ5m4+Xp5a -dt+EGqjYX8qdsbKJ6ZqwvvrOZMk4vDBCgYwtPEPGLYCJUNpcNcaFH013eRxW9pQ7 -X9azRrFBfbiIGvucgoqfA6N4mnA/hiHZFAO/37n3+R5aKpy+SGVPlPfj4nE9jEQ0 -bi4R2kECgYEA0vV263GC2PApybf0+UPi1B+i4MtoV4/bUT2CKByEKt++uKgH0bIT -ytNNobnEoMSyrwkRGNlsFD0xRZn7yQBFuAZ0H0pk1QZsHVhy8Ax1S9tC2ec3qloH -OxvWwkTbjlCkdWTEa3Hgp9R+ITChs4vLUVYkEEIeH+w4uv4uUI/iLYcCgYEAxSkN -3No9nPJgK4OhR1e+K9MaFGVApdV7hgNFWcGpOs6NY4Br9Exdr5MNY8ONIVXmFBSO -D1z3yC/Wv1lFHpJKoJB6lSaPLMfYoyRODR27X3/9WyPbbEQmj1LPn7yFOrxu+Rtw -PvwtoWz22C+iY2a1OCtYlQRt75uFuUy/R+YDS1kCgYAY7jtUjcCZ5J/7n8rKnnQy -/14fVMqK9oxDqJI2LVCNRI3kgq8R4AqowJn/D9Yl7bj0KI0ls6QrLCQjhpQT5/1V -RyX4ALGz3yNtwGMs7Tv7b/kRiVqxvhv2p06U/NA5nFwjJ3py/C1BiEIzp+ykLFQl -05lFdprayO7XYmyUxrxDBwKBgHR4+FOMKfHZza5nKJO45gOFxlZeKn7NSxe0Pvfz -TEINW8F4UE4V/xKF0Ncq4ujakf0y12mEBm0gtcB8wDR7P9LzBzete898kCJwhBuD -gPm95rI80Jd4+z6YMUewWxSD1Rscdob++wXuFTVJjqkiN9Ri5wXBhCCUw77f8/BJ -lZChAoGAZdXu+to1Y86TxO+kdZmIk40suEBrmdyAS7M33Zwby/8I+YA9wuU2qTF7 -VyKUz6LGmAH1WdPgFeDFFjwbzahLTJ6wsTYnoCT7RB881v1gKliIp71M6vKzEPd+ -73l0v89CEMXeLNMWkTp32QTVXp/wu6Nh6B9Rh2gPGWrZLi0eRIg= ------END RSA PRIVATE KEY----- +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCqtRND1GELHgVf +RSNBcvIeZ6o3ARieCKi7UGS13c8GAYJjpHId9FEOT6K2HhqsdsgAgmNK8LcY5NWM +yNascwZN3oW+ftTiXTB5EhOgdOaOnREiVbvQcm6RIrQJQGp+Lbv3i3KUZzQ9ziTN +u0OturKD2HADfZMMBCySTSZocws+tY3KykGrMeXsKuF+5WKlxBCJM4qWrC3izhzN +LGsocwIkiqTXIaW79Tp0GpAB58j3QgR5rBCIcbDDtnAoLOuyQTkHakiCKT+kx6Dp +SbtptgNseeAuZAm8AGIrifSEEiOSPrvLNsemdkyORK+lupyp4mulXRyyGp8WofnH +5qCGsrB/AgMBAAECggEAEeha9AOkhoEIuTzMfQIs6gmf7pM73OA1qyi81+ND43Jn +FkLbAc9ubVznScmQrMTtKDiFaAgmfQMMb+9lrcOHRsXwK9KsTG7QNfMygeOhiBZs +BLkPscc/WBRW5J2QTlpH3KkRRQYqkc09SSO67s+mCHS9E3Cp61nIvzQ+sjhXVqWT +MqsT8vMkAvfYPd8S03PMbVnio1qa2fBAVAV9OQR0Emxy3Jhs1j7hwJcW+Za3LCLX +349Vwfpm9Z43BvLP3JQpy9PwR9w3shQpHh2KawPEcR/87OZGt4J0Lazce9E6ehlz +p1fKzXot+Ib0mGCKxBR18q6W/dD8sUbh20bOphfVAQKBgQDVfmPHaS1uf514zEY2 +EaUsUEYg7w49PRsTIejoVl3rQ3zitEY4G8SAKvygXdqoaz/oZyitAIpNPfWhArAW +S8Jl701U77HDklG/7rOpHu86ukWbf78mbpi5Pevl8eky2Y+KK6J+HDrkIxXszlwK +5SGKRzwbXhYzKcfV1Kv1JaNo/wKBgQDMseV3wxNJuyBfgsxtdkVl/EuCaHR9AhQ2 ++bo/qiPLXkwn5sjBt7v0W1TWtFXIv8Ql40CmPyzfx4sXHkUPgwR9OjtB/9k/JHNz +fmDgyMcHEi76lgX6NfCNZ9zn7nrGXxioCupCtzbZLmpowQ9JsWtY7aezhrvnR4na +lqwOlJ04gQKBgHeRMZCc9zTLhAYDYc8mUrteuef3+qGRd3hfpXF57V+1jFvrWuhR +pNujN5kY9z0b087093LsuohICykBuFzEUTuFqmnHVGeTqvUzWqt9B758VD+ibAu6 +JQ6Z/2Po+BHihuF7TqYSaWkbvUvCxwnTCdXHz4IET6qX27OGSEgjrEFBAoGAWs7o +79J48hKz7+tE6aK4cuAUV4wjcXnEogbzm17NdmElA+acVWQ/W85QutdR3zdDu1Mv +0RDhxXzutZjWJmWHTk7RDxyDskFnfmsoa1BnCzoD4WNAz5myV+kl1Ta0/OAfPh1p +g6NoD71Js6NCNv0glFkKNtik75uLCpAhUG6ZuQECgYEAyXGiSLsuMzdhW/IzInji +kJ6kg1Ux6/BDyKSk2vJ0NS2QldexXlFQdqYLIgMJcdJHiY/9uQZd47zG2fbLLqBr +OoyqmRqf9PkTvDJMSD/dkznryGWBvvTHsCU8CKxOnvp89sExR751kMdtZiJ/yxvw +lTptjiYN7dVJ9kMewH/WJh4= +-----END PRIVATE KEY----- diff --git a/integration-tests/certs/cacert.pem b/integration-tests/certs/cacert.pem index fd9f36efc9..1e9d949636 100644 --- a/integration-tests/certs/cacert.pem +++ b/integration-tests/certs/cacert.pem @@ -1,83 +1,35 @@ -Certificate: - Data: - Version: 3 (0x2) - Serial Number: 12259044356400093384 (0xaa20dfe414fc18c8) - Signature Algorithm: sha256WithRSAEncryption - Issuer: C=US, ST=CA, O=Apache, OU=Apache Pulsar, CN=localhost - Validity - Not Before: Apr 13 08:01:48 2021 GMT - Not After : Apr 12 08:01:48 2024 GMT - Subject: C=US, ST=CA, O=Apache, OU=Apache Pulsar, CN=localhost - Subject Public Key Info: - Public Key Algorithm: rsaEncryption - Public-Key: (2048 bit) - Modulus: - 00:c6:d2:63:e1:12:01:3d:7f:6f:4a:50:fb:25:f4: - eb:2e:73:e7:a8:4e:cc:0a:8d:a4:cb:30:b7:89:7a: - 69:14:c4:0c:94:a1:4a:b8:39:24:1b:fc:1c:b6:c6: - 39:2a:94:26:69:a7:d4:13:59:e8:35:b5:34:d9:a4: - 36:b6:03:c4:ad:4f:a4:78:94:3c:6b:94:99:70:b5: - 4f:6c:ee:5b:ca:ce:15:2d:ab:38:04:8f:52:0d:48: - d5:ce:74:08:00:f2:69:ae:59:08:b6:75:3d:13:c9: - 33:32:dc:47:de:fd:1e:bb:c7:50:a1:b1:ca:84:43: - c4:7d:d9:fd:85:dc:93:d1:b3:59:58:e2:85:c6:25: - d2:bf:3a:7c:6f:8a:04:ae:c8:45:be:74:d7:80:d2: - b7:03:3c:11:71:92:9e:e3:f8:37:a1:94:11:12:ee: - 2c:88:5a:f8:55:27:1d:fd:6f:22:41:33:9f:e0:d5: - e5:13:30:64:1f:63:a6:1c:f2:7f:e1:1c:e5:a8:3a: - 63:45:7b:1b:5c:4e:6a:1b:c9:0e:23:ae:08:3c:b7: - 17:d4:07:3c:0f:89:54:4b:58:f2:36:d7:53:10:9a: - d2:5f:d9:82:ac:fd:6e:80:6b:30:4c:96:bd:39:bd: - 9f:28:49:31:1d:08:23:5c:18:a0:c3:ce:25:f3:49: - 0c:0d - Exponent: 65537 (0x10001) - X509v3 extensions: - X509v3 Basic Constraints: - CA:TRUE - X509v3 Authority Key Identifier: - DirName:/C=US/ST=CA/O=Apache/OU=Apache Pulsar/CN=localhost - serial:AA:20:DF:E4:14:FC:18:C8 - - X509v3 Subject Key Identifier: - 59:A0:A4:78:56:E1:56:61:4E:DC:E9:80:FF:EC:66:B6:2F:04:6D:AA - X509v3 Subject Alternative Name: - DNS:localhost - Signature Algorithm: sha256WithRSAEncryption - 1d:d1:d0:86:49:a8:ca:a7:ec:13:5d:0c:60:9a:36:ba:55:17: - 04:2a:4b:29:4b:7e:11:68:e1:a9:f5:53:72:a1:04:1e:ee:63: - 36:2c:0a:fd:4f:e0:c3:64:e2:b4:d3:03:76:80:83:4e:88:3f: - f1:18:25:62:e1:82:cf:37:ca:31:95:48:db:a0:58:9f:ae:54: - 7b:ea:fa:5f:f1:b5:fa:8e:29:b9:d6:7e:de:ee:78:82:f3:bf: - 20:37:f2:dc:f2:88:46:ae:16:59:a4:7e:67:50:a4:4d:91:bc: - 39:c2:65:0e:a6:97:5e:ed:18:2e:45:99:67:d3:66:21:93:28: - ba:7d:19:a9:47:88:74:2b:ef:27:5d:32:41:ab:96:c5:cc:12: - 07:ec:f9:72:9c:a9:e5:3e:88:40:f6:11:82:b3:0b:6a:86:95: - db:f5:32:a9:e9:9f:fd:1d:67:53:27:08:19:d3:8e:77:2f:74: - 42:0b:e9:31:13:20:1e:75:5c:c3:24:1f:68:60:a8:90:bc:22: - 28:3a:92:35:e6:1d:f0:e6:fb:11:23:13:50:bc:fd:46:42:90: - d1:00:9c:a6:1b:d8:d6:58:e7:a6:9a:a9:4e:5b:93:37:e2:9b: - fa:fb:89:a6:d6:13:f8:a0:fb:84:b2:5f:74:30:7b:5e:51:5d: - e9:4e:74:17 -----BEGIN CERTIFICATE----- -MIID6zCCAtOgAwIBAgIJAKog3+QU/BjIMA0GCSqGSIb3DQEBCwUAMFcxCzAJBgNV -BAYTAlVTMQswCQYDVQQIDAJDQTEPMA0GA1UECgwGQXBhY2hlMRYwFAYDVQQLDA1B -cGFjaGUgUHVsc2FyMRIwEAYDVQQDDAlsb2NhbGhvc3QwHhcNMjEwNDEzMDgwMTQ4 -WhcNMjQwNDEyMDgwMTQ4WjBXMQswCQYDVQQGEwJVUzELMAkGA1UECAwCQ0ExDzAN -BgNVBAoMBkFwYWNoZTEWMBQGA1UECwwNQXBhY2hlIFB1bHNhcjESMBAGA1UEAwwJ -bG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAxtJj4RIB -PX9vSlD7JfTrLnPnqE7MCo2kyzC3iXppFMQMlKFKuDkkG/wctsY5KpQmaafUE1no -NbU02aQ2tgPErU+keJQ8a5SZcLVPbO5bys4VLas4BI9SDUjVznQIAPJprlkItnU9 -E8kzMtxH3v0eu8dQobHKhEPEfdn9hdyT0bNZWOKFxiXSvzp8b4oErshFvnTXgNK3 -AzwRcZKe4/g3oZQREu4siFr4VScd/W8iQTOf4NXlEzBkH2OmHPJ/4RzlqDpjRXsb -XE5qG8kOI64IPLcX1Ac8D4lUS1jyNtdTEJrSX9mCrP1ugGswTJa9Ob2fKEkxHQgj -XBigw84l80kMDQIDAQABo4G5MIG2MAwGA1UdEwQFMAMBAf8wcQYDVR0jBGowaKFb -pFkwVzELMAkGA1UEBhMCVVMxCzAJBgNVBAgMAkNBMQ8wDQYDVQQKDAZBcGFjaGUx -FjAUBgNVBAsMDUFwYWNoZSBQdWxzYXIxEjAQBgNVBAMMCWxvY2FsaG9zdIIJAKog -3+QU/BjIMB0GA1UdDgQWBBRZoKR4VuFWYU7c6YD/7Ga2LwRtqjAUBgNVHREEDTAL -gglsb2NhbGhvc3QwDQYJKoZIhvcNAQELBQADggEBAB3R0IZJqMqn7BNdDGCaNrpV -FwQqSylLfhFo4an1U3KhBB7uYzYsCv1P4MNk4rTTA3aAg06IP/EYJWLhgs83yjGV -SNugWJ+uVHvq+l/xtfqOKbnWft7ueILzvyA38tzyiEauFlmkfmdQpE2RvDnCZQ6m -l17tGC5FmWfTZiGTKLp9GalHiHQr7yddMkGrlsXMEgfs+XKcqeU+iED2EYKzC2qG -ldv1Mqnpn/0dZ1MnCBnTjncvdEIL6TETIB51XMMkH2hgqJC8Iig6kjXmHfDm+xEj -E1C8/UZCkNEAnKYb2NZY56aaqU5bkzfim/r7iabWE/ig+4SyX3Qwe15RXelOdBc= +MIIGDDCCA/SgAwIBAgIJAJnACyGkFss5MA0GCSqGSIb3DQEBCwUAMIGSMQswCQYD +VQQGEwJVUzETMBEGA1UECAwKQ2FsaWZvcm5pYTEjMCEGA1UECgwaQXBhY2hlIFNv +ZnR3YXJlIEZvdW5kYXRpb24xDzANBgNVBAsMBlB1bHNhcjESMBAGA1UEAwwJUHVs +c2FyIENBMSQwIgYJKoZIhvcNAQkBFhVkZXZAcHVsc2FyLmFwYWNoZS5vcmcwHhcN +MjEwNjIyMDU1NTM5WhcNNDEwNjE3MDU1NTM5WjCBkjELMAkGA1UEBhMCVVMxEzAR +BgNVBAgMCkNhbGlmb3JuaWExIzAhBgNVBAoMGkFwYWNoZSBTb2Z0d2FyZSBGb3Vu +ZGF0aW9uMQ8wDQYDVQQLDAZQdWxzYXIxEjAQBgNVBAMMCVB1bHNhciBDQTEkMCIG +CSqGSIb3DQEJARYVZGV2QHB1bHNhci5hcGFjaGUub3JnMIICIjANBgkqhkiG9w0B +AQEFAAOCAg8AMIICCgKCAgEAlvc1vIIKOAFomJh+kKJ1nWID5cLISNOEaRdLzTZo +kOofm6uoWa8HkjxmuPmbnqGGxjnjnffMvuo8df6aj1VFokhVgUUYf3yN0gtlVCyd +tWa9OMgWrKDoVqHcLCnl8giBuaC1RhDmmvYgHZDr260X/AfEaPmYq426bLAQYBxZ +fqNu4sGHxbCEciFi0DvnZzWT++rs2+hgrPmqQFbLzojx4m4riJW1IWnIvKwnf/j8 +V+rWZy/DJDG/oQLl6eIyZ6oy8nxn+3PYq6NKavdeayLwT1HuckX1ooHA+/a+r+6J +m1mEg5OwwQOP720gmgjC/8P3/xKyM6e0s01vKnllf1PbIwTrpzcFspaXzeEfAFpV +ykyf/HTgTcid+Mj3zHL5isgN3qj5FjSG+CVjTpnO5mGtbXQI+jMmbKPU3ubFsVw3 +hrm6ZVDP4P+JvJBzs40usVNHJVtSZYJbGojx/tadQahpZuFPFuV8O5eYNwrRlC6M +sKVX6IPT+lqXKQLzTtQNzxPDe8SJi+ca+5mQ1kf+VyQ5AkHM+cR/Pn49jFn3beA2 +GIVWoAS7SzUO5Sjle+NI1UnnY/6ZU6OnlsO5sCLp7nY1jWoirZQU+kX6/Wi4z2MH +7d6Nw0NZS87h3iNuTgVxgTDVVx7VNISeX9fi1mCkel4XPZ3VaSH9HDxlR0SihvtH +9IcCAwEAAaNjMGEwHQYDVR0OBBYEFJwSlUAQswluhzWF9O2pNXmyvc4IMB8GA1Ud +IwQYMBaAFJwSlUAQswluhzWF9O2pNXmyvc4IMA8GA1UdEwEB/wQFMAMBAf8wDgYD +VR0PAQH/BAQDAgGGMA0GCSqGSIb3DQEBCwUAA4ICAQAZgrrJdHgiQwWkJZFJtk+1 +iouKKlaz64Yx8Txk8Q49TJwDpCSmL8zk7H92oVlxtoA2IpSBAFIDyrGov/3/EaLc +6oyXYi2AKsYpTSZuLj2KP8HfB+TG8LHpER4Pv+WtX532V2A008hKyxSRUDWgI2tt +F85GqpopwqDmDRsa06qCoBustrbDHDrjEWCld81qPZWOh07k7NRlX9t5D0a0sIOq +lQaLnlI08fyxAK0GsJnvQIZ9/UVz0sB9OfEE2PHRSGAbAPzb49SpUrncLIV4rHTH +DswtKEvaEpcmuXGIb+HI0ZgZnS56oiu83NbvSCMMSgqHvXdu65g8SYGUihy8NoaV +Icr0jFfgbb21bhWgy7VyTnMbCRWEtJknPGzmi6/U9wnAmIYP0L9AXjbXkvysDmdd +VoKT8HddPvgfZC7sgamMh0JxkzOHsO+C7DXw6kLp8FhoQPGk2/mK8McBfDY3g3hu +meynM7GW6/7gkYXmuf1g3os+5iBHN4Rg0GL7RuXPrdypNKeVIrrq+D3b81LWuON3 +cFOKSX1R3Hto5mM3Zrt522KzzwJdIaEKyTazbdQWJXOuIHGGsk7CroJJxCkjSvSK +U23R3i6Sld5HUYAGr0iBsf++7YNi3Z86KPovZHyUPdgUpW3Vk+jw9FJnWlkQYQxn +VmnNXgp4FahF1fz8lTAddw== -----END CERTIFICATE----- diff --git a/integration-tests/certs/client-cert.pem b/integration-tests/certs/client-cert.pem index d4c481166a..af546097b9 100644 --- a/integration-tests/certs/client-cert.pem +++ b/integration-tests/certs/client-cert.pem @@ -1,81 +1,31 @@ -Certificate: - Data: - Version: 3 (0x2) - Serial Number: 9806047019225563731 (0x881613cf2fb7c253) - Signature Algorithm: sha1WithRSAEncryption - Issuer: C=US, ST=CA, O=Apache, OU=Apache Pulsar, CN=localhost - Validity - Not Before: Apr 13 08:01:48 2021 GMT - Not After : Apr 11 08:01:48 2031 GMT - Subject: C=US, ST=CA, O=Apache, OU=Apache Pulsar, CN=localhost - Subject Public Key Info: - Public Key Algorithm: rsaEncryption - Public-Key: (2048 bit) - Modulus: - 00:a8:90:05:5f:14:7a:de:41:98:14:82:fe:f2:c3: - 4a:81:1b:47:91:36:83:5e:9d:b2:78:7b:26:50:1c: - b9:45:8f:de:b5:61:2e:3a:6d:86:e3:e4:64:2b:13: - 4b:52:34:89:20:2d:0b:40:b8:76:aa:4f:cc:b5:dc: - 4d:c7:dc:11:32:e9:84:7f:51:b0:ae:1d:c8:70:7e: - 16:e4:cf:da:c3:56:4a:c8:9f:93:4a:7f:fb:fd:32: - 20:0a:bd:78:3a:b5:65:cf:fb:12:a2:ca:c2:71:a0: - ac:9d:c2:7b:94:b3:b8:18:c0:2b:f4:ab:ce:72:d4: - 7a:00:3f:b3:4a:be:97:ea:7c:8c:8e:7a:50:58:22: - 0c:00:3a:26:8c:47:8e:64:dd:b5:86:8f:c9:bf:61: - 48:00:55:81:7e:f4:5b:12:54:00:b2:7e:15:c0:b7: - 76:0c:88:9a:7a:89:8b:2b:be:f8:04:af:71:5a:43: - ca:e3:9e:d1:71:ff:10:65:69:db:c3:c7:58:cb:ac: - b2:34:34:be:ef:26:67:de:48:71:3d:ba:a4:6e:6e: - fa:27:8b:97:37:ab:e2:f7:d1:ee:3f:58:e0:65:f5: - 04:d8:05:ac:49:a5:c0:88:6b:91:5f:c8:79:aa:28: - c9:36:e2:12:b0:c1:71:16:c2:38:b0:6f:b8:6f:f8: - fa:d9 - Exponent: 65537 (0x10001) - X509v3 extensions: - X509v3 Basic Constraints: - CA:FALSE - X509v3 Authority Key Identifier: - keyid:59:A0:A4:78:56:E1:56:61:4E:DC:E9:80:FF:EC:66:B6:2F:04:6D:AA - - X509v3 Subject Key Identifier: - E0:D9:A8:7D:49:EC:BF:DF:1D:88:45:9B:D7:E5:BE:5A:90:C6:98:59 - X509v3 Subject Alternative Name: - DNS:localhost - Signature Algorithm: sha1WithRSAEncryption - 73:02:b5:ce:ba:8d:2c:34:1b:04:1d:40:00:1f:71:01:aa:b5: - 2f:db:b3:31:35:a0:74:f2:ca:24:e5:df:18:f2:85:a3:be:cc: - 62:4d:67:b5:ff:db:4e:33:38:4b:58:69:34:8b:cc:91:22:a4: - aa:cf:c1:fe:61:46:c6:f7:f3:cf:65:b4:d3:47:b8:07:be:bb: - dc:8d:ad:c4:c9:cb:23:6b:64:14:78:1e:16:a5:af:ca:28:d4: - 8c:b5:ec:31:76:74:41:91:a0:50:6b:66:99:8d:d5:a6:3a:4e: - 7b:5b:00:74:85:22:e7:6f:82:d8:ba:42:0f:71:44:3c:48:27: - a9:d5:af:86:e7:4e:a3:11:83:92:12:a5:da:46:d1:9b:c3:a8: - 70:f7:18:cb:1a:91:55:6c:04:bb:4b:7c:b3:49:89:71:39:cd: - e0:f4:41:58:bb:db:66:97:4a:c2:2f:ca:65:b6:e5:d2:dc:00: - 9f:7e:35:68:ef:08:40:c8:f1:a8:45:f3:5e:4b:93:9d:ee:82: - 19:3a:27:a8:a1:32:95:89:0c:a9:00:ee:ae:a9:02:69:f3:0b: - 1f:42:3a:ed:67:64:f4:88:fe:03:7e:93:90:f3:a3:d2:45:73: - 77:8d:2a:42:65:b9:d7:a7:29:9a:df:da:23:c9:2a:c4:f9:8f: - a4:66:13:72 -----BEGIN CERTIFICATE----- -MIIDlDCCAnygAwIBAgIJAIgWE88vt8JTMA0GCSqGSIb3DQEBBQUAMFcxCzAJBgNV -BAYTAlVTMQswCQYDVQQIDAJDQTEPMA0GA1UECgwGQXBhY2hlMRYwFAYDVQQLDA1B -cGFjaGUgUHVsc2FyMRIwEAYDVQQDDAlsb2NhbGhvc3QwHhcNMjEwNDEzMDgwMTQ4 -WhcNMzEwNDExMDgwMTQ4WjBXMQswCQYDVQQGEwJVUzELMAkGA1UECAwCQ0ExDzAN -BgNVBAoMBkFwYWNoZTEWMBQGA1UECwwNQXBhY2hlIFB1bHNhcjESMBAGA1UEAwwJ -bG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAqJAFXxR6 -3kGYFIL+8sNKgRtHkTaDXp2yeHsmUBy5RY/etWEuOm2G4+RkKxNLUjSJIC0LQLh2 -qk/MtdxNx9wRMumEf1Gwrh3IcH4W5M/aw1ZKyJ+TSn/7/TIgCr14OrVlz/sSosrC -caCsncJ7lLO4GMAr9KvOctR6AD+zSr6X6nyMjnpQWCIMADomjEeOZN21ho/Jv2FI -AFWBfvRbElQAsn4VwLd2DIiaeomLK774BK9xWkPK457Rcf8QZWnbw8dYy6yyNDS+ -7yZn3khxPbqkbm76J4uXN6vi99HuP1jgZfUE2AWsSaXAiGuRX8h5qijJNuISsMFx -FsI4sG+4b/j62QIDAQABo2MwYTAJBgNVHRMEAjAAMB8GA1UdIwQYMBaAFFmgpHhW -4VZhTtzpgP/sZrYvBG2qMB0GA1UdDgQWBBTg2ah9Sey/3x2IRZvX5b5akMaYWTAU -BgNVHREEDTALgglsb2NhbGhvc3QwDQYJKoZIhvcNAQEFBQADggEBAHMCtc66jSw0 -GwQdQAAfcQGqtS/bszE1oHTyyiTl3xjyhaO+zGJNZ7X/204zOEtYaTSLzJEipKrP -wf5hRsb3889ltNNHuAe+u9yNrcTJyyNrZBR4Hhalr8oo1Iy17DF2dEGRoFBrZpmN -1aY6TntbAHSFIudvgti6Qg9xRDxIJ6nVr4bnTqMRg5ISpdpG0ZvDqHD3GMsakVVs -BLtLfLNJiXE5zeD0QVi722aXSsIvymW25dLcAJ9+NWjvCEDI8ahF815Lk53ughk6 -J6ihMpWJDKkA7q6pAmnzCx9COu1nZPSI/gN+k5Dzo9JFc3eNKkJludenKZrf2iPJ -KsT5j6RmE3I= +MIIFZDCCA0ygAwIBAgICEAEwDQYJKoZIhvcNAQELBQAwgZIxCzAJBgNVBAYTAlVT +MRMwEQYDVQQIDApDYWxpZm9ybmlhMSMwIQYDVQQKDBpBcGFjaGUgU29mdHdhcmUg +Rm91bmRhdGlvbjEPMA0GA1UECwwGUHVsc2FyMRIwEAYDVQQDDAlQdWxzYXIgQ0Ex +JDAiBgkqhkiG9w0BCQEWFWRldkBwdWxzYXIuYXBhY2hlLm9yZzAeFw0yMTA2MjIw +NTU1MzlaFw0zMTA2MjAwNTU1MzlaMIGOMQswCQYDVQQGEwJVUzETMBEGA1UECAwK +Q2FsaWZvcm5pYTEjMCEGA1UECgwaQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24x +DzANBgNVBAsMBlB1bHNhcjEOMAwGA1UEAwwFYWRtaW4xJDAiBgkqhkiG9w0BCQEW +FWRldkBwdWxzYXIuYXBhY2hlLm9yZzCCASIwDQYJKoZIhvcNAQEBBQADggEPADCC +AQoCggEBAM6DhhMRpatXj/+6TgI8nAH3gbx6O8onimBAb99SyoJUfxpSePMjQQh2 +C+SwDc/SxWHYoer38gRvPDmObrPc0Nsa17+ppmpGPGG/9WQFiVKyXXEABgQAjRMc +OJJBTEojdZ2WffWhlulo/7GOD/hSCZ8nb/7DIk0MWtZ3XwKrEnoeojrC7cjLM1HT +odtYW45Fq/NsC/YBl5w0muhTsI6jHC54hUGllvD/GIRzSCfbXFVR23oHgdgoFWc8 +YTcd1pP7nzq3ZhNTLLvXprJDW8lhExHiZlUIUTViZZmfaBfXiOw1tJvvsZ+r1CUh +QJoOGW4R7tj6VpENyey7pH2Er7jiTQUCAwEAAaOBxTCBwjAJBgNVHRMEAjAAMBEG +CWCGSAGG+EIBAQQEAwIFoDAzBglghkgBhvhCAQ0EJhYkT3BlblNTTCBHZW5lcmF0 +ZWQgQ2xpZW50IENlcnRpZmljYXRlMB0GA1UdDgQWBBRZgXK4DZrHLR7CmORj2ies +IC9lRDAfBgNVHSMEGDAWgBScEpVAELMJboc1hfTtqTV5sr3OCDAOBgNVHQ8BAf8E +BAMCBeAwHQYDVR0lBBYwFAYIKwYBBQUHAwIGCCsGAQUFBwMEMA0GCSqGSIb3DQEB +CwUAA4ICAQAPuXzTjW+84f3boomOvGNS57lSPyx83SNXXvtXW4VudkhCMc2pggkT +OY2bizRVny5uGE7eHo7a7dBoujHiAk+5lN7KNlY/VZpX6qRg0Ps5JUTfZSFYSdL+ +ZXFhOwFgDqlLVWlUXaJ4PiABjcZks4tWZ66Yg+BNI2MU5PJGySOoAuntM53x4hA7 +xHmyX7nsmqUDA7C4TbUKr+R9KmPhPQhJwcC1mFwhJtIbVoxn4E7lSr6BsNQuZ6aJ +aDjh8qxqJpmL9dYqle9fuP1naJfKOLcOdAudAgIsjpQTWbGn8lPUGIke488NlECq +LYEi47GLMrW4CcD4Z4lvDRKCtO6HYsv+AOuyN0O3iALOyg8+khqDufqBZQWSq8GI +jpCDd+0zNzHCDEmSMgzo8wmA9LeRpEzH6d3v35gjJB8yFC3IygQrKGIqkF/FkTuw +ZcZhK8Tjb/5wrgSq8jm9pQbvm6iMTbor+E7bOS+9jepCTc0C/TgLyxxcJu4GraGA +wjOY7MhVFPkz1agp0gj6YvtMlPDS8AgQjn4RxW1aCWuqWH0p5g1XOoGyDqWorhX+ +uzr5LjBjAOX931MFG0nHNefaLEfWaooKDH0HiAfpHzfvOwes4j47WquFzpPR+n4d +Sr1cdPEqBHWmqLtQyZqnAgB3c1vgwpig907lHY/L2t38xwvFAENY8A== -----END CERTIFICATE----- diff --git a/integration-tests/certs/client-key.pem b/integration-tests/certs/client-key.pem index d4ad41e7ae..dadac1dbfc 100644 --- a/integration-tests/certs/client-key.pem +++ b/integration-tests/certs/client-key.pem @@ -1,27 +1,28 @@ ------BEGIN RSA PRIVATE KEY----- -MIIEpAIBAAKCAQEAqJAFXxR63kGYFIL+8sNKgRtHkTaDXp2yeHsmUBy5RY/etWEu -Om2G4+RkKxNLUjSJIC0LQLh2qk/MtdxNx9wRMumEf1Gwrh3IcH4W5M/aw1ZKyJ+T -Sn/7/TIgCr14OrVlz/sSosrCcaCsncJ7lLO4GMAr9KvOctR6AD+zSr6X6nyMjnpQ -WCIMADomjEeOZN21ho/Jv2FIAFWBfvRbElQAsn4VwLd2DIiaeomLK774BK9xWkPK -457Rcf8QZWnbw8dYy6yyNDS+7yZn3khxPbqkbm76J4uXN6vi99HuP1jgZfUE2AWs -SaXAiGuRX8h5qijJNuISsMFxFsI4sG+4b/j62QIDAQABAoIBAEhOp482yNIojyVV -YCohLged/SBEoyvmN9XI9hFtkOhPI/jofB0iOLfvlCyKNUhJMv1zdR1JQ9YhLy5g -F5mBRHtzgpvfHgxCyGXBIO78XtpmXOQc1+Egb6cYZnhU+Isun0MNb9wXYDu/8ewm -rj6GrZUocnKddIz7qm4K5U1N3vr9ygjpzY4ltpLagZFD+kY8l4eh9ayUwx7qIRv1 -036TRajcDabN/T8dGwQ/87lUQtSPcmTPRNXNRePRLCFC96J8cLE2cGG8iQdEpSrJ -ehzpkXWBShGY5NjJEhgIdyVGMOMfealsRjQH9mX3wyLV7JSE/2NSvBIJPBc4arwW -if+dvvUCgYEA28ErIxPL+rNl7qdzjGAQFFtESjeBUhrUYulbJk1TOz914D87oRCo -GDEGnQdMNgXDe4ddPiDKk9ukSsCxR7Ow+7Xtag/9BHLZers1DfAniEdLEo51fr2H -CrAazeEaUsvg0ThNRrZWd17a2RLX4Npqa4JQFddsb+TJnUqmSQmIBocCgYEAxF1X -kYHmcDeObzTZAJGmDCVHyggxsx2BMS2gIbmasuPQVqBBq7kilzywuZ/5BqFRSMmI -kaZ/vRvCLCD0amFLgVbrHPoX9ZqsHhsQyyeWPTUX1XgBEMkO5nUYV3QPPIABPE/j -PD1yW82elmueJXzaxNHrCrXlvGmh04OSCGZJ658CgYEAyr2hfaAzDV1bil2k5fpu -e0lkbJBe2A/2qDagqGqogj3fE39jFQrX+lGox1DHSAPzQNE2/i2kl9sI7QpvxDYM -73z9x19u0B7hdDCpFu3cmwcjRMB3t/FFWF/2qsCl14NPK9ckvDtW9JRnleOjlUgM -7JjyJ+Ryn5zVQS9w0GthupsCgYEAkzF9AL+k4x1iEY5F1B1ZoOUZKSJ3TtKhEyIm -VOFySwJmG2lJkNXYThHaT2rRGt01tyAK1VM/kZwLM1UIoe2I0c07tJ7r8tkch/y1 -2/MytXLlCMlJ0zFDkBDaTEPdX/Z79p/TsEZHQPWjGRJWm7c5rivEymSLb26nNYv8 -nXs/kfUCgYBRl7KEAHwtjc6d8Hqe+bfSNRe6tiWxXfXeZD9OATzTDDnkvl9jChwD -B8Vj1hR/lzvG3SRs9uS+ehYfnRxQ/ZQJjCFtUy9Q/bfUZ2ajNj4PL5v0IV6pmeE9 -Ek3E8iazpIcNmNvg+S6MB0Cf2BrLssc5RHkDnbAAuKEtqfqpn/pAVA== ------END RSA PRIVATE KEY----- +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQDOg4YTEaWrV4// +uk4CPJwB94G8ejvKJ4pgQG/fUsqCVH8aUnjzI0EIdgvksA3P0sVh2KHq9/IEbzw5 +jm6z3NDbGte/qaZqRjxhv/VkBYlSsl1xAAYEAI0THDiSQUxKI3Wdln31oZbpaP+x +jg/4UgmfJ2/+wyJNDFrWd18CqxJ6HqI6wu3IyzNR06HbWFuORavzbAv2AZecNJro +U7COoxwueIVBpZbw/xiEc0gn21xVUdt6B4HYKBVnPGE3HdaT+586t2YTUyy716ay +Q1vJYRMR4mZVCFE1YmWZn2gX14jsNbSb77Gfq9QlIUCaDhluEe7Y+laRDcnsu6R9 +hK+44k0FAgMBAAECggEAf2VZl3fieJHazkozB2Q2ZWw3R3ssWSgNUBVQLjdXoBDa +G/XWkT8lRj/0VE5bJuKCSzsln9r3HM82GteH6UMGQLQNdjoE5DiR1Udi7SIRNce8 +pxH5pAxm+uwE5gyMMIwxaVbGO3pEq5bPbcwShACv2NOuXj4ocdl2TCl1niLEkjop +X7Kj8jswsBb/zF8ItmW0rEpGoZXWyToh06fknONsAe0z0spXwZTHRFVxVn8oVmuN +RwF3Fjjtdfsi9BE+jJEzrpOzfZSgjT44hbWSfe7LVEfLVQN4CqyKo2mLzWAC6hWZ +/2o5KmJeGkR4ylTOXSmb92a7dGN218KjPgrI/C3fQQKBgQD06kxfkn3cT7wuIWv0 +hcSLKYGPlSh3UxiC8F4V3FFtGwAmfiXXnBgDslWtxyp1b9UxSRSR2OmGELVFzFW2 +XpB4VTE8jAu2ktpAnjpzaCg2izhFQYeo4rK/ebA46Pl9/A8tE5Jm+JWDf7Il6J7m ++MSswi4OuMm6WPm6cm6MugiUUQKBgQDX3EnG+UkFm36o0ybL5PJz+TEQVbNg0MM4 +8O0cev0zqvOQAiQs9Afu5A5oeZKvruyPoYKCu4XJBh5+va+/7JNnSFv1W0h2EOKQ +mULm7TDs+FGFOcgDcnnRX4n6KJ5nRaFj8jxMObwmOtlBSdZuWoem4Hh2tTMDbkmX +hOwI00FEdQKBgERB3GEyeaNiEYclx2mRsnI6ciJrJqkC3qz3grFDAuqOgN85fyB1 +H3hARoK1JE8wiNFAldwEASldFLhMN4MpHch7rco7GbSbfvpNXur1F2gEKv2aQs29 +yKV2q7dgrs/+5NfiQYYzN7A7cZIBH/AtWtrvD1kJBSfLTmY5KDhOTL/RAoGBAKQc +5TKToILU3403S83EuNiXYkgLAB/hWCtYcM0wbIT1ucuI9TwAJG/pj1K6610KXuws +XO9L5sEZF+JAEPiXwtHJX7DAv+yTlc1d2FF9j676c/AYKAsG47rdKie47gWSGMvG +BTXmxzdBkp9ipdasmWm0S6HkzW4UUXsTIejrxDlpAoGAF1aDHgC0ZCjNEuUYiyOM +oRkANY2DmjpQjQnX6zDmq+6+HITDM0g76PQS2igvB06Ysym+o997b1FcDot1ml4L +9/10z9TcRVZiKbbF7e9h75QvmtGv2rR6Ucc+yOpJUsBX+vSpnQlUGT6y9qrF0dr/ +YjVcFUPAwA0nqNZvQfdcH14= +-----END PRIVATE KEY----- diff --git a/integration-tests/certs/generate.sh b/integration-tests/certs/generate.sh index 0b587c0c6f..9294e0342c 100755 --- a/integration-tests/certs/generate.sh +++ b/integration-tests/certs/generate.sh @@ -1,16 +1,82 @@ -#/bin/sh +#!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# -openssl genrsa -out ca.key 2048 -openssl genrsa -out client-key.pem 2048 -openssl genrsa -out broker-key.pem 2048 +export CA_HOME=$(pwd) +echo $CA_HOME -openssl req -sha256 -x509 -new -text -out cacert.pem -key ca.key -config openssl.cnf -extensions v3_ca -days 1095 -openssl req -sha256 -new -text -out client.csr -key client-key.pem -config openssl.cnf -extensions v3_req -openssl req -sha256 -new -text -out broker.csr -key broker-key.pem -config openssl.cnf -extensions v3_req +if [ -d "certs" ]; then + rm -rf certs +fi -openssl x509 -req -extfile openssl.cnf -extensions v3_req -in client.csr -out client.crt -CA cacert.pem -CAkey ca.key -CAcreateserial -days 3650 -openssl x509 -in client.crt -text > client-cert.pem -openssl x509 -req -extfile openssl.cnf -extensions v3_req -in broker.csr -out broker.crt -CA cacert.pem -CAkey ca.key -CAcreateserial -days 3650 -openssl x509 -in broker.crt -text > broker-cert.pem +if [ -d "crl" ]; then + rm -rf crl +fi -rm *.crt *.csr *.key *.srl \ No newline at end of file +if [ -d "newcerts" ]; then + rm -rf newcerts +fi + +if [ -d "private" ]; then + rm -rf private +fi + +if [ -d "index.txt" ]; then + rm -rf index.txt +fi + +if [ -d "serial" ]; then + rm -rf serial +fi + +mkdir certs crl newcerts private +chmod 700 private/ +touch index.txt +echo 1000 > serial +openssl genrsa -out private/ca.key.pem 4096 + +openssl req -config openssl.cnf -key private/ca.key.pem \ + -new -x509 -days 7300 -sha256 -extensions v3_ca \ + -out certs/ca.cert.pem -subj '/C=US/ST=California/O=Apache Software Foundation/OU=Pulsar/CN=Pulsar CA/emailAddress=dev@pulsar.apache.org' + +openssl genrsa -out private/broker.key.pem 2048 +openssl pkcs8 -topk8 -inform PEM -outform PEM \ + -in private/broker.key.pem -out private/broker.key-pk8.pem -nocrypt +openssl req -config openssl.cnf \ + -key private/broker.key.pem -new -sha256 -out crl/broker.csr.pem -subj '/C=US/ST=California/O=Apache Software Foundation/OU=Pulsar/CN=localhost/emailAddress=dev@pulsar.apache.org' +yes | openssl ca -config openssl.cnf -extensions server_cert \ + -days 3650 -notext -md sha256 \ + -in crl/broker.csr.pem -out certs/broker.cert.pem + +openssl genrsa -out private/client.key.pem 2048 +openssl pkcs8 -topk8 -inform PEM -outform PEM \ + -in private/client.key.pem -out private/client.key-pk8.pem -nocrypt +openssl req -config openssl.cnf \ + -key private/client.key.pem -new -sha256 -out crl/client.csr.pem -subj '/C=US/ST=California/O=Apache Software Foundation/OU=Pulsar/CN=admin/emailAddress=dev@pulsar.apache.org' +yes | openssl ca -config openssl.cnf -extensions usr_cert \ + -days 3650 -notext -md sha256 \ + -in crl/client.csr.pem -out certs/client.cert.pem + +mv certs/ca.cert.pem cacert.pem +mv certs/broker.cert.pem broker-cert.pem +mv private/broker.key-pk8.pem broker-key.pem +mv certs/client.cert.pem client-cert.pem +mv private/client.key-pk8.pem client-key.pem + +rm -rf certs crl newcerts private index.txt* serial* \ No newline at end of file diff --git a/integration-tests/certs/openssl.cnf b/integration-tests/certs/openssl.cnf index 82dbbd94ca..9e30ed0f36 100644 --- a/integration-tests/certs/openssl.cnf +++ b/integration-tests/certs/openssl.cnf @@ -1,19 +1,156 @@ -[req] -distinguished_name = req_distinguished_name -prompt = no -[req_distinguished_name] -C = US -ST = CA -O = Apache -OU = Apache Pulsar -CN = localhost -[v3_req] +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# +# +# Configuration file for testing certificate authority. +# The environment variable, CA_HOME, must be set to point to the directory +# containing this file before running any openssl commands. +# + +CA_HOME = ./ +CA_HOME = ${ENV::CA_HOME} +[ ca ] +# `man ca` +default_ca = CA_default + +[ CA_default ] +# Directory and file locations. +dir = ${ENV::CA_HOME} +certs = $dir/certs +crl_dir = $dir/crl +new_certs_dir = $dir/newcerts +database = $dir/index.txt +serial = $dir/serial +RANDFILE = $dir/private/.rand + +# The root key and root certificate. +private_key = $dir/private/ca.key.pem +certificate = $dir/certs/ca.cert.pem + +# For certificate revocation lists. +crlnumber = $dir/crlnumber +crl = $dir/crl/ca.crl.pem +crl_extensions = crl_ext +default_crl_days = 30 + +# SHA-1 is deprecated, so use SHA-2 instead. +default_md = sha256 + +name_opt = ca_default +cert_opt = ca_default +default_days = 375 +preserve = no +policy = policy_strict + +[ policy_strict ] +# The root CA should only sign intermediate certificates that match. +# See the POLICY FORMAT section of `man ca`. +countryName = match +stateOrProvinceName = match +organizationName = match +organizationalUnitName = optional +commonName = supplied +emailAddress = optional + +[ policy_loose ] +# Allow the intermediate CA to sign a more diverse range of certificates. +# See the POLICY FORMAT section of the `ca` man page. +countryName = optional +stateOrProvinceName = optional +localityName = optional +organizationName = optional +organizationalUnitName = optional +commonName = supplied +emailAddress = optional + +[ req ] +# Options for the `req` tool (`man req`). +default_bits = 2048 +distinguished_name = req_distinguished_name +string_mask = utf8only + +# SHA-1 is deprecated, so use SHA-2 instead. +default_md = sha256 + +# Extension to add when the -x509 option is used. +x509_extensions = v3_ca + +[ req_distinguished_name ] +# See . +countryName = Country Name (2 letter code) +stateOrProvinceName = State or Province Name +localityName = Locality Name +0.organizationName = Organization Name +organizationalUnitName = Organizational Unit Name +commonName = Common Name +emailAddress = Email Address + +# Optionally, specify some defaults. +countryName_default = US +stateOrProvinceName_default = California +localityName_default = Palo Alto +0.organizationName_default = Apache Software Foundation +organizationalUnitName_default = Pulsar +emailAddress_default = dev@pulsar.apache.org + +[ v3_ca ] +# Extensions for a typical CA (`man x509v3_config`). +subjectKeyIdentifier = hash +authorityKeyIdentifier = keyid:always,issuer +basicConstraints = critical, CA:true +keyUsage = critical, digitalSignature, cRLSign, keyCertSign + +[ v3_intermediate_ca ] +# Extensions for a typical intermediate CA (`man x509v3_config`). +subjectKeyIdentifier = hash +authorityKeyIdentifier = keyid:always,issuer +basicConstraints = critical, CA:true, pathlen:0 +keyUsage = critical, digitalSignature, cRLSign, keyCertSign + +[ usr_cert ] +# Extensions for client certificates (`man x509v3_config`). basicConstraints = CA:FALSE -authorityKeyIdentifier=keyid,issuer -subjectKeyIdentifier=hash +nsCertType = client, email +nsComment = "OpenSSL Generated Client Certificate" +subjectKeyIdentifier = hash +authorityKeyIdentifier = keyid,issuer +keyUsage = critical, nonRepudiation, digitalSignature, keyEncipherment +extendedKeyUsage = clientAuth, emailProtection + +[ server_cert ] +# Extensions for server certificates (`man x509v3_config`). +basicConstraints = CA:FALSE +nsCertType = server +nsComment = "OpenSSL Generated Server Certificate" +subjectKeyIdentifier = hash +authorityKeyIdentifier = keyid,issuer:always +keyUsage = critical, digitalSignature, keyEncipherment +extendedKeyUsage = serverAuth subjectAltName=DNS:localhost -[v3_ca] -basicConstraints = CA:TRUE -authorityKeyIdentifier=keyid,issuer -subjectKeyIdentifier=hash -subjectAltName=DNS:localhost \ No newline at end of file + +[ crl_ext ] +# Extension for CRLs (`man x509v3_config`). +authorityKeyIdentifier=keyid:always + +[ ocsp ] +# Extension for OCSP signing certificates (`man ocsp`). +basicConstraints = CA:FALSE +subjectKeyIdentifier = hash +authorityKeyIdentifier = keyid,issuer +keyUsage = critical, digitalSignature +extendedKeyUsage = critical, OCSPSigning \ No newline at end of file diff --git a/integration-tests/standalone.conf b/integration-tests/standalone.conf index b426ff32b7..a298a61428 100644 --- a/integration-tests/standalone.conf +++ b/integration-tests/standalone.conf @@ -22,9 +22,6 @@ # Zookeeper quorum connection string zookeeperServers= -# Deprecated. Global zookeeper quorum connection string -globalZookeeperServers= - # Configuration Store connection string configurationStoreServers= @@ -110,7 +107,7 @@ tokenSecretKey=file:///pulsar/tokens/secret.key # Role names that are treated as "super-user", meaning they will be able to do all admin # operations and publish/consume from all topics -superUserRoles=localhost,superUser +superUserRoles=localhost,superUser,admin # Authentication settings of the broker itself. Used when the broker connects to other brokers, # either in same or other clusters @@ -279,5 +276,19 @@ defaultRetentionSizeInMB=0 # How often to check whether the connections are still alive keepAliveIntervalSeconds=30 -# How often broker checks for inactive topics to be deleted (topics with no subscriptions and no one connected) +# Enable topic auto creation if new producer or consumer connected (disable auto creation with value false) +allowAutoTopicCreation=true + +# The type of topic that is allowed to be automatically created.(partitioned/non-partitioned) +allowAutoTopicCreationType=non-partitioned + +# Enable subscription auto creation if new consumer connected (disable auto creation with value false) +allowAutoSubscriptionCreation=true + +### --- Deprecated config variables --- ### + +# Deprecated. Global zookeeper quorum connection string +globalZookeeperServers= + +# Deprecated. Use brokerDeleteInactiveTopicsFrequencySeconds brokerServicePurgeInactiveFrequencyInSeconds=60 diff --git a/pulsar-test-service-start.sh b/pulsar-test-service-start.sh index c0bdaddfdb..a7c4797615 100755 --- a/pulsar-test-service-start.sh +++ b/pulsar-test-service-start.sh @@ -28,6 +28,7 @@ IMAGE_NAME=pulsar-client-go-test:latest if [[ -f /.dockerenv ]]; then # When running tests inside docker PULSAR_ADMIN=/pulsar/bin/pulsar-admin + cat /pulsar/conf/standalone.conf /pulsar/bin/pulsar-daemon start standalone --no-functions-worker --no-stream-storage else docker build -t ${IMAGE_NAME} . diff --git a/pulsar/internal/http_client.go b/pulsar/internal/http_client.go index b60aa89c4a..c160623c61 100644 --- a/pulsar/internal/http_client.go +++ b/pulsar/internal/http_client.go @@ -52,7 +52,7 @@ func (c *httpClient) Close() { } type HTTPClient interface { - Get(endpoint string, obj interface{}) error + Get(endpoint string, obj interface{}, params map[string]string) error Closable } @@ -143,8 +143,8 @@ func (c *httpClient) MakeRequest(method, endpoint string) (*http.Response, error return resp, nil } -func (c *httpClient) Get(endpoint string, obj interface{}) error { - _, err := c.GetWithQueryParams(endpoint, obj, nil, true) +func (c *httpClient) Get(endpoint string, obj interface{}, params map[string]string) error { + _, err := c.GetWithQueryParams(endpoint, obj, params, true) if _, ok := err.(*url.Error); ok { // We can retry this kind of requests over a connection error because they're // not specific to a particular broker. @@ -156,7 +156,7 @@ func (c *httpClient) Get(endpoint string, obj interface{}) error { retryTime = backoff.Next() c.log.Debugf("Retrying httpRequest in {%v} with timeout in {%v}", retryTime, c.requestTimeout) time.Sleep(retryTime) - _, err = c.GetWithQueryParams(endpoint, obj, nil, true) + _, err = c.GetWithQueryParams(endpoint, obj, params, true) if _, ok := err.(*url.Error); ok { continue } else { diff --git a/pulsar/internal/lookup_service.go b/pulsar/internal/lookup_service.go index 867644f90e..e4dac1a079 100644 --- a/pulsar/internal/lookup_service.go +++ b/pulsar/internal/lookup_service.go @@ -291,7 +291,7 @@ func (h *httpLookupService) Lookup(topic string) (*LookupResult, error) { } lookupData := &httpLookupData{} - err = h.httpClient.Get(basePath+GetTopicRestPath(topicName), lookupData) + err = h.httpClient.Get(basePath+GetTopicRestPath(topicName), lookupData, nil) if err != nil { return nil, err } @@ -327,7 +327,7 @@ func (h *httpLookupService) GetPartitionedTopicMetadata(topic string) (*Partitio tMetadata := &PartitionedTopicMetadata{} - err = h.httpClient.Get(path, tMetadata) + err = h.httpClient.Get(path, tMetadata, map[string]string{"checkAllowAutoCreation": "true"}) if err != nil { return nil, err } @@ -348,7 +348,7 @@ func (h *httpLookupService) GetTopicsOfNamespace(namespace string, mode GetTopic topics := []string{} - err := h.httpClient.Get(path, &topics) + err := h.httpClient.Get(path, &topics, nil) if err != nil { return nil, err } diff --git a/pulsar/internal/lookup_service_test.go b/pulsar/internal/lookup_service_test.go index 92255a60d3..16043199b9 100644 --- a/pulsar/internal/lookup_service_test.go +++ b/pulsar/internal/lookup_service_test.go @@ -531,7 +531,7 @@ type MockHTTPClient struct { func (c *MockHTTPClient) Close() {} -func (c *MockHTTPClient) Get(endpoint string, obj interface{}) error { +func (c *MockHTTPClient) Get(endpoint string, obj interface{}, params map[string]string) error { if strings.Contains(endpoint, HTTPLookupServiceBasePathV1) || strings.Contains(endpoint, HTTPLookupServiceBasePathV2) { return mockHTTPGetLookupResult(obj) From 98338e544d31570504cc06f4da63da47cbb8e628 Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Thu, 24 Jun 2021 16:36:49 +0800 Subject: [PATCH 08/25] fix resend pendingItems race condition (#551) --- pulsar/producer_partition.go | 42 ++++++++++++++++++++++++++++-------- 1 file changed, 33 insertions(+), 9 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index b365f83995..19aa06ea02 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -231,10 +231,29 @@ func (p *partitionProducer) grabCnx() error { p.log.WithField("cnx", res.Cnx.ID()).Debug("Connected producer") pendingItems := p.pendingQueue.ReadableSlice() - if len(pendingItems) > 0 { - p.log.Infof("Resending %d pending batches", len(pendingItems)) - for _, pi := range pendingItems { - p.cnx.WriteData(pi.(*pendingItem).batchData) + viewSize := len(pendingItems) + if viewSize > 0 { + p.log.Infof("Resending %d pending batches", viewSize) + lastViewItem := pendingItems[viewSize-1].(*pendingItem) + + // iterate at most pending items + for i := 0; i < viewSize; i++ { + item := p.pendingQueue.Poll() + if item == nil { + continue + } + pi := item.(*pendingItem) + // when resending pending batches, we update the sendAt timestamp and put to the back of queue + // to avoid pending item been removed by failTimeoutMessages and cause race condition + pi.Lock() + pi.sentAt = time.Now() + pi.Unlock() + p.pendingQueue.Put(pi) + p.cnx.WriteData(pi.batchData) + + if pi == lastViewItem { + break + } } } return nil @@ -523,8 +542,7 @@ func (p *partitionProducer) failTimeoutMessages() { } // flag the send has completed with error, flush make no effect - pi.completed = true - buffersPool.Put(pi.batchData) + pi.Complete() pi.Unlock() // finally reached the last view item, current iteration ends @@ -706,9 +724,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) } // Mark this pending item as done - pi.completed = true - // Return buffer to the pool since we're now done using it - buffersPool.Put(pi.batchData) + pi.Complete() } func (p *partitionProducer) internalClose(req *closeProducer) { @@ -800,3 +816,11 @@ type flushRequest struct { waitGroup *sync.WaitGroup err error } + +func (i *pendingItem) Complete() { + if i.completed { + return + } + i.completed = true + buffersPool.Put(i.batchData) +} From f33b594034c9433faae78a144983ab3c8f785add Mon Sep 17 00:00:00 2001 From: Yuto Furuta Date: Thu, 24 Jun 2021 18:07:47 +0900 Subject: [PATCH 09/25] support DisableReplication (#543) Co-authored-by: Yuto Furuta --- pulsar/message.go | 3 +++ pulsar/producer_partition.go | 5 +++++ 2 files changed, 8 insertions(+) diff --git a/pulsar/message.go b/pulsar/message.go index 2a4343f352..23dfefb418 100644 --- a/pulsar/message.go +++ b/pulsar/message.go @@ -48,6 +48,9 @@ type ProducerMessage struct { // ReplicationClusters override the replication clusters for this message. ReplicationClusters []string + // Disable the replication for this message + DisableReplication bool + // SequenceID set the sequence id to assign to the current message SequenceID *int64 diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 19aa06ea02..7dd176c1bf 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -411,6 +411,11 @@ func (p *partitionProducer) internalSend(request *sendRequest) { if !sendAsBatch { p.internalFlushCurrentBatch() } + + if msg.DisableReplication { + msg.ReplicationClusters = []string{"__local__"} + } + added := p.batchBuilder.Add(smm, p.sequenceIDGenerator, payload, request, msg.ReplicationClusters, deliverAt) if !added { From 96ce2de573ea8ad14173f41d6e48b6b48a8b0aa0 Mon Sep 17 00:00:00 2001 From: dferstay Date: Thu, 24 Jun 2021 02:27:12 -0700 Subject: [PATCH 10/25] Fix data race while accessing connection in partitionConsumer (#535) The partitionConsumer maintains a few internal go-routines, two of which access the underlying internal.Connection. The main runEvenstLoop() go-routine reads the connection field while a separate go-routine is used to detect connnection loss, initiate reconnection, and sets the connection. Previously, access to the conn field was not synchronized. Now, the conn field is read and written atomically; resolving the data race. Signed-off-by: Daniel Ferstay Co-authored-by: Daniel Ferstay --- pulsar/consumer_partition.go | 43 ++++++++++++++++++++++++------------ 1 file changed, 29 insertions(+), 14 deletions(-) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 94a914db49..daaf759f92 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -107,7 +107,7 @@ type partitionConsumer struct { state atomic.Int32 options *partitionConsumerOpts - conn internal.Connection + conn atomic.Value topic string name string @@ -238,7 +238,7 @@ func (pc *partitionConsumer) internalUnsubscribe(unsub *unsubscribeRequest) { RequestId: proto.Uint64(requestID), ConsumerId: proto.Uint64(pc.consumerID), } - _, err := pc.client.rpcClient.RequestOnCnx(pc.conn, requestID, pb.BaseCommand_UNSUBSCRIBE, cmdUnsubscribe) + _, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), requestID, pb.BaseCommand_UNSUBSCRIBE, cmdUnsubscribe) if err != nil { pc.log.WithError(err).Error("Failed to unsubscribe consumer") unsub.err = err @@ -248,7 +248,7 @@ func (pc *partitionConsumer) internalUnsubscribe(unsub *unsubscribeRequest) { return } - pc.conn.DeleteConsumeHandler(pc.consumerID) + pc._getConn().DeleteConsumeHandler(pc.consumerID) if pc.nackTracker != nil { pc.nackTracker.Close() } @@ -276,7 +276,7 @@ func (pc *partitionConsumer) requestGetLastMessageID() (trackingMessageID, error RequestId: proto.Uint64(requestID), ConsumerId: proto.Uint64(pc.consumerID), } - res, err := pc.client.rpcClient.RequestOnCnx(pc.conn, requestID, + res, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), requestID, pb.BaseCommand_GET_LAST_MESSAGE_ID, cmdGetLastMessageID) if err != nil { pc.log.WithError(err).Error("Failed to get last message id") @@ -326,7 +326,7 @@ func (pc *partitionConsumer) internalRedeliver(req *redeliveryRequest) { } } - pc.client.rpcClient.RequestOnCnxNoWait(pc.conn, + pc.client.rpcClient.RequestOnCnxNoWait(pc._getConn(), pb.BaseCommand_REDELIVER_UNACKNOWLEDGED_MESSAGES, &pb.CommandRedeliverUnacknowledgedMessages{ ConsumerId: proto.Uint64(pc.consumerID), MessageIds: msgIDDataList, @@ -399,7 +399,7 @@ func (pc *partitionConsumer) requestSeekWithoutClear(msgID messageID) error { MessageId: id, } - _, err = pc.client.rpcClient.RequestOnCnx(pc.conn, requestID, pb.BaseCommand_SEEK, cmdSeek) + _, err = pc.client.rpcClient.RequestOnCnx(pc._getConn(), requestID, pb.BaseCommand_SEEK, cmdSeek) if err != nil { pc.log.WithError(err).Error("Failed to reset to message id") return err @@ -435,7 +435,7 @@ func (pc *partitionConsumer) internalSeekByTime(seek *seekByTimeRequest) { MessagePublishTime: proto.Uint64(uint64(seek.publishTime.UnixNano() / int64(time.Millisecond))), } - _, err := pc.client.rpcClient.RequestOnCnx(pc.conn, requestID, pb.BaseCommand_SEEK, cmdSeek) + _, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), requestID, pb.BaseCommand_SEEK, cmdSeek) if err != nil { pc.log.WithError(err).Error("Failed to reset to message publish time") seek.err = err @@ -465,7 +465,7 @@ func (pc *partitionConsumer) internalAck(req *ackRequest) { AckType: pb.CommandAck_Individual.Enum(), } - pc.client.rpcClient.RequestOnCnxNoWait(pc.conn, pb.BaseCommand_ACK, cmdAck) + pc.client.rpcClient.RequestOnCnxNoWait(pc._getConn(), pb.BaseCommand_ACK, cmdAck) } func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, headersAndPayload internal.Buffer) error { @@ -607,7 +607,7 @@ func (pc *partitionConsumer) internalFlow(permits uint32) error { ConsumerId: proto.Uint64(pc.consumerID), MessagePermits: proto.Uint32(permits), } - pc.client.rpcClient.RequestOnCnxNoWait(pc.conn, pb.BaseCommand_FLOW, cmdFlow) + pc.client.rpcClient.RequestOnCnxNoWait(pc._getConn(), pb.BaseCommand_FLOW, cmdFlow) return nil } @@ -843,7 +843,7 @@ func (pc *partitionConsumer) internalClose(req *closeRequest) { ConsumerId: proto.Uint64(pc.consumerID), RequestId: proto.Uint64(requestID), } - _, err := pc.client.rpcClient.RequestOnCnx(pc.conn, requestID, pb.BaseCommand_CLOSE_CONSUMER, cmdClose) + _, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), requestID, pb.BaseCommand_CLOSE_CONSUMER, cmdClose) if err != nil { pc.log.WithError(err).Warn("Failed to close consumer") } else { @@ -855,7 +855,7 @@ func (pc *partitionConsumer) internalClose(req *closeRequest) { } pc.setConsumerState(consumerClosed) - pc.conn.DeleteConsumeHandler(pc.consumerID) + pc._getConn().DeleteConsumeHandler(pc.consumerID) if pc.nackTracker != nil { pc.nackTracker.Close() } @@ -971,9 +971,9 @@ func (pc *partitionConsumer) grabConn() error { pc.name = res.Response.ConsumerStatsResponse.GetConsumerName() } - pc.conn = res.Cnx + pc._setConn(res.Cnx) pc.log.Info("Connected consumer") - pc.conn.AddConsumeHandler(pc.consumerID, pc) + pc._getConn().AddConsumeHandler(pc.consumerID, pc) msgType := res.Response.GetType() @@ -1104,7 +1104,7 @@ func (pc *partitionConsumer) discardCorruptedMessage(msgID *pb.MessageIdData, "validationError": validationError, }).Error("Discarding corrupted message") - pc.client.rpcClient.RequestOnCnxNoWait(pc.conn, + pc.client.rpcClient.RequestOnCnxNoWait(pc._getConn(), pb.BaseCommand_ACK, &pb.CommandAck{ ConsumerId: proto.Uint64(pc.consumerID), MessageId: []*pb.MessageIdData{msgID}, @@ -1113,6 +1113,21 @@ func (pc *partitionConsumer) discardCorruptedMessage(msgID *pb.MessageIdData, }) } +// _setConn sets the internal connection field of this partition consumer atomically. +// Note: should only be called by this partition consumer when a new connection is available. +func (pc *partitionConsumer) _setConn(conn internal.Connection) { + pc.conn.Store(conn) +} + +// _getConn returns internal connection field of this partition consumer atomically. +// Note: should only be called by this partition consumer before attempting to use the connection +func (pc *partitionConsumer) _getConn() internal.Connection { + // Invariant: The conn must be non-nill for the lifetime of the partitionConsumer. + // For this reason we leave this cast unchecked and panic() if the + // invariant is broken + return pc.conn.Load().(internal.Connection) +} + func convertToMessageIDData(msgID trackingMessageID) *pb.MessageIdData { if msgID.Undefined() { return nil From 641cb9da88695f5dcac5cc7028577a519df4d21a Mon Sep 17 00:00:00 2001 From: Deen Date: Thu, 24 Jun 2021 17:28:51 +0800 Subject: [PATCH 11/25] updating comments to conform to golang comment specification (#532) Co-authored-by: ronaldoliu --- pulsar/client.go | 27 ++++++++++++++++----------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/pulsar/client.go b/pulsar/client.go index b9c8db80a0..cc6fb3cef1 100644 --- a/pulsar/client.go +++ b/pulsar/client.go @@ -25,18 +25,20 @@ import ( "github.com/apache/pulsar-client-go/pulsar/log" ) +// NewClient Creates a pulsar client instance func NewClient(options ClientOptions) (Client, error) { return newClient(options) } -// Opaque interface that represents the authentication credentials +// Authentication Opaque interface that represents the authentication credentials type Authentication interface{} +// NewAuthentication Creates an authentication by name and params func NewAuthentication(name string, params string) (Authentication, error) { return auth.NewProvider(name, params) } -// Create new Authentication provider with specified auth token +// NewAuthenticationToken Creates new Authentication provider with specified auth token func NewAuthenticationToken(token string) Authentication { return auth.NewAuthenticationToken(token) } @@ -49,32 +51,34 @@ func NewAuthenticationTokenFromSupplier(tokenSupplier func() (string, error)) Au return auth.NewAuthenticationTokenFromSupplier(tokenSupplier) } -// Create new Authentication provider with specified auth token from a file +// NewAuthenticationTokenFromFile Creates new Authentication provider with specified auth token from a file func NewAuthenticationTokenFromFile(tokenFilePath string) Authentication { return auth.NewAuthenticationTokenFromFile(tokenFilePath) } -// Create new Authentication provider with specified TLS certificate and private key +// NewAuthenticationTLS Creates new Authentication provider with specified TLS certificate and private key func NewAuthenticationTLS(certificatePath string, privateKeyPath string) Authentication { return auth.NewAuthenticationTLS(certificatePath, privateKeyPath) } -// Create new Authentication provider with specified TLS certificate supplier +// NewAuthenticationFromTLSCertSupplier Create new Authentication provider with specified TLS certificate supplier func NewAuthenticationFromTLSCertSupplier(tlsCertSupplier func() (*tls.Certificate, error)) Authentication { return auth.NewAuthenticationFromTLSCertSupplier(tlsCertSupplier) } +// NewAuthenticationAthenz Creates Athenz Authentication provider func NewAuthenticationAthenz(authParams map[string]string) Authentication { athenz, _ := auth.NewAuthenticationAthenzWithParams(authParams) return athenz } +// NewAuthenticationOAuth2 Creates OAuth2 Authentication provider func NewAuthenticationOAuth2(authParams map[string]string) Authentication { oauth, _ := auth.NewAuthenticationOAuth2WithParams(authParams) return oauth } -// Builder interface that is used to construct a Pulsar Client instance. +// ClientOptions is used to construct a Pulsar Client instance. type ClientOptions struct { // Configure the service URL for the Pulsar service. // This parameter is required @@ -117,22 +121,23 @@ type ClientOptions struct { CustomMetricsLabels map[string]string } +// Client represents a pulsar client type Client interface { - // Create the producer instance + // CreateProducer Creates the producer instance // This method will block until the producer is created successfully CreateProducer(ProducerOptions) (Producer, error) - // Create a `Consumer` by subscribing to a topic. + // Subscribe Creates a `Consumer` by subscribing to a topic. // // If the subscription does not exist, a new subscription will be created and all messages published after the // creation will be retained until acknowledged, even if the consumer is not connected Subscribe(ConsumerOptions) (Consumer, error) - // Create a Reader instance. + // CreateReader Creates a Reader instance. // This method will block until the reader is created successfully. CreateReader(ReaderOptions) (Reader, error) - // Fetch the list of partitions for a given topic + // TopicPartitions Fetches the list of partitions for a given topic // // If the topic is partitioned, this will return a list of partition names. // If the topic is not partitioned, the returned list will contain the topic @@ -142,6 +147,6 @@ type Client interface { // {@link Consumer} or {@link Producer} instances directly on a particular partition. TopicPartitions(topic string) ([]string, error) - // Close the Client and free associated resources + // Close Closes the Client and free associated resources Close() } From 0f3e504604aa00c208d3f5946436db2e4837b7fd Mon Sep 17 00:00:00 2001 From: dferstay Date: Fri, 25 Jun 2021 16:36:38 -0700 Subject: [PATCH 12/25] Producer respects Context passed to Send() and SendAsync() when applying backpressure (#534) Previously, the Producer ignored the context passed to Send() and SendAsync(). Now, the Producer respects the context in the case where the ProducerOptions.MaxPendingMessages limit is reached. In this case, the producer will block until a permit for sending the message is available or until the context expires, whichever comes first. Failures to send messages due to context expiration are communicated to callers via the existing TimeoutError error code. Signed-off-by: Daniel Ferstay Co-authored-by: Daniel Ferstay --- pulsar/internal/semaphore.go | 16 ++++++-- pulsar/internal/semaphore_test.go | 20 +++++++++- pulsar/producer_partition.go | 6 ++- pulsar/producer_test.go | 66 +++++++++++++++++++++++++++++++ 4 files changed, 101 insertions(+), 7 deletions(-) diff --git a/pulsar/internal/semaphore.go b/pulsar/internal/semaphore.go index a34497f865..6a67cc3550 100644 --- a/pulsar/internal/semaphore.go +++ b/pulsar/internal/semaphore.go @@ -18,6 +18,7 @@ package internal import ( + "context" "sync/atomic" log "github.com/sirupsen/logrus" @@ -26,7 +27,7 @@ import ( type Semaphore interface { // Acquire a permit, if one is available and returns immediately, // reducing the number of available permits by one. - Acquire() + Acquire(ctx context.Context) bool // Try to acquire a permit. The method will return immediately // with a `true` if it was possible to acquire a permit and @@ -63,14 +64,21 @@ func NewSemaphore(maxPermits int32) Semaphore { } } -func (s *semaphore) Acquire() { +func (s *semaphore) Acquire(ctx context.Context) bool { permits := atomic.AddInt32(&s.permits, 1) if permits <= s.maxPermits { - return + return true } // Block on the channel until a new permit is available - <-s.ch + // or the context expires + select { + case <-s.ch: + return true + case <-ctx.Done(): + atomic.AddInt32(&s.permits, -1) + return false + } } func (s *semaphore) TryAcquire() bool { diff --git a/pulsar/internal/semaphore_test.go b/pulsar/internal/semaphore_test.go index 0de69fcf66..b692d6864f 100644 --- a/pulsar/internal/semaphore_test.go +++ b/pulsar/internal/semaphore_test.go @@ -18,6 +18,7 @@ package internal import ( + "context" "sync" "testing" "time" @@ -35,7 +36,7 @@ func TestSemaphore(t *testing.T) { for i := 0; i < n; i++ { go func() { - s.Acquire() + assert.True(t, s.Acquire(context.Background())) time.Sleep(100 * time.Millisecond) s.Release() wg.Done() @@ -48,7 +49,7 @@ func TestSemaphore(t *testing.T) { func TestSemaphore_TryAcquire(t *testing.T) { s := NewSemaphore(1) - s.Acquire() + assert.True(t, s.Acquire(context.Background())) assert.False(t, s.TryAcquire()) @@ -58,3 +59,18 @@ func TestSemaphore_TryAcquire(t *testing.T) { assert.False(t, s.TryAcquire()) s.Release() } + +func TestSemaphore_ContextExpire(t *testing.T) { + s := NewSemaphore(1) + + assert.True(t, s.Acquire(context.Background())) + + ctx, cancel := context.WithCancel(context.Background()) + cancel() + assert.False(t, s.Acquire(ctx)) + + assert.False(t, s.TryAcquire()) + s.Release() + + assert.True(t, s.TryAcquire()) +} diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 7dd176c1bf..002d261cfd 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -48,6 +48,7 @@ var ( errFailAddToBatch = newError(AddToBatchFailed, "message add to batch failed") errSendTimeout = newError(TimeoutError, "message send timeout") errSendQueueIsFull = newError(ProducerQueueIsFull, "producer send queue is full") + errContextExpired = newError(TimeoutError, "message send context expired") errMessageTooLarge = newError(MessageTooBig, "message size exceeds MaxMessageSize") buffersPool sync.Pool @@ -658,7 +659,10 @@ func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *Producer return } } else { - p.publishSemaphore.Acquire() + if !p.publishSemaphore.Acquire(ctx) { + callback(nil, msg, errContextExpired) + return + } } p.metrics.MessagesPending.Inc() diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 4d62cac169..7c3dbd76bb 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -930,6 +930,72 @@ func TestSendTimeout(t *testing.T) { makeHTTPCall(t, http.MethodDelete, quotaURL, "") } +func TestSendContextExpired(t *testing.T) { + quotaURL := adminURL + "/admin/v2/namespaces/public/default/backlogQuota" + quotaFmt := `{"limit": "%d", "policy": "producer_request_hold"}` + makeHTTPCall(t, http.MethodPost, quotaURL, fmt.Sprintf(quotaFmt, 1024)) + + client, err := NewClient(ClientOptions{ + URL: serviceURL, + }) + assert.NoError(t, err) + defer client.Close() + + topicName := newTopicName() + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topicName, + SubscriptionName: "send_context_expired_sub", + }) + assert.Nil(t, err) + defer consumer.Close() // subscribe but do nothing + + noRetry := uint(0) + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topicName, + MaxPendingMessages: 1, + SendTimeout: 2 * time.Second, + MaxReconnectToBroker: &noRetry, + }) + assert.Nil(t, err) + defer producer.Close() + + // first send completes and fills the available backlog + id, err := producer.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }) + assert.Nil(t, err) + assert.NotNil(t, id) + + // waiting for the backlog check + time.Sleep((5 + 1) * time.Second) + + // next publish will not complete due to the backlog quota being full; + // this consumes the only available MaxPendingMessages permit + wg := sync.WaitGroup{} + wg.Add(1) + producer.SendAsync(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }, func(_ MessageID, _ *ProducerMessage, _ error) { + // we're not interested in the result of this send, but we don't + // want to exit this test case until it completes + wg.Done() + }) + + // final publish will block waiting for a send permit to become available + // then fail when the ctx times out + ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) + defer cancel() + id, err = producer.Send(ctx, &ProducerMessage{ + Payload: make([]byte, 1024), + }) + assert.NotNil(t, err) + assert.Nil(t, id) + + wg.Wait() + + makeHTTPCall(t, http.MethodDelete, quotaURL, "") +} + type noopProduceInterceptor struct{} func (noopProduceInterceptor) BeforeSend(producer Producer, message *ProducerMessage) {} From ea3e0540dbb59596b6e9cf4b6dbced8d5f9b0d11 Mon Sep 17 00:00:00 2001 From: xiaolong ran Date: Wed, 30 Jun 2021 20:02:45 +0800 Subject: [PATCH 13/25] Fix channel data race (#558) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: xiaolongran ### Motivation In `internalSendRequest`, We will add the request to be sent to the `pendingReqs` map, even when the current connection status is `connectionClosed`, we will append the request, which will cause the current request's callback to be called twice First: ``` func (c *connection) internalSendRequest(req *request) { c.pendingLock.Lock() if req.id != nil { c.pendingReqs[*req.id] = req } c.pendingLock.Unlock() if c.getState() == connectionClosed { c.log.Warnf("internalSendRequest failed for connectionClosed") // In Here, call req.callback ************* if req.callback != nil { req.callback(req.cmd, ErrConnectionClosed) } } else { c.writeCommand(req.cmd) } } ``` Twice: ``` func (c *connection) run() { // All reads come from the reader goroutine go c.reader.readFromConnection() go c.runPingCheck() c.log.Debugf("Connection run starting with request capacity=%d queued=%d", cap(c.incomingRequestsCh), len(c.incomingRequestsCh)) defer func() { // all the accesses to the pendingReqs should be happened in this run loop thread, // including the final cleanup, to avoid the issue https://github.com/apache/pulsar-client-go/issues/239 c.pendingLock.Lock() for id, req := range c.pendingReqs { // In Here, call req.callback ********** req.callback(nil, errors.New("connection closed")) delete(c.pendingReqs, id) } c.pendingLock.Unlock() c.Close() }() .... } ``` In fact, when the current connection is in the `connectionClosed` state, we don’t need to append the request to the `pendingReqs` map, so we don’t need to process the request when it’s closed. ### Modifications When the connection is closed, the current request to be sent is not added to the `pendingReqs` map. --- pulsar/internal/connection.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 15dfceca81..1e3ce0264b 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -584,17 +584,17 @@ func (c *connection) SendRequestNoWait(req *pb.BaseCommand) error { } func (c *connection) internalSendRequest(req *request) { - c.pendingLock.Lock() - if req.id != nil { - c.pendingReqs[*req.id] = req - } - c.pendingLock.Unlock() if c.getState() == connectionClosed { c.log.Warnf("internalSendRequest failed for connectionClosed") if req.callback != nil { req.callback(req.cmd, ErrConnectionClosed) } } else { + c.pendingLock.Lock() + if req.id != nil { + c.pendingReqs[*req.id] = req + } + c.pendingLock.Unlock() c.writeCommand(req.cmd) } } From 1a9f356c125e2b15b69aa8c7a5f6707fb9ce3e51 Mon Sep 17 00:00:00 2001 From: Denis Vergnes Date: Wed, 30 Jun 2021 05:10:47 -0700 Subject: [PATCH 14/25] Add jitter to exponential backoff (#556) (#557) Co-authored-by: Denis Vergnes Fixes #556 ### Motivation When reconnecting to a broker, an exponential backoff strategy is used. However, there is no randomness involved which means in case of a general outage impacting several clients, they will try to reconnect at the same time. ### Modifications The modification consists of adding a jitter that adds 20% of delay after the delay is doubled on each retry. --- pulsar/internal/backoff.go | 18 +++++++--- pulsar/internal/backoff_test.go | 59 +++++++++++++++++++++++++++++++++ 2 files changed, 72 insertions(+), 5 deletions(-) create mode 100644 pulsar/internal/backoff_test.go diff --git a/pulsar/internal/backoff.go b/pulsar/internal/backoff.go index c558f580d9..f172bdccd2 100644 --- a/pulsar/internal/backoff.go +++ b/pulsar/internal/backoff.go @@ -18,20 +18,27 @@ package internal import ( + "math/rand" "time" ) -// Backoff +func init() { + rand.Seed(time.Now().UnixNano()) +} + +// Backoff computes the delay before retrying an action. +// It uses an exponential backoff with jitter. The jitter represents up to 20 percents of the delay. type Backoff struct { backoff time.Duration } const ( - minBackoff = 100 * time.Millisecond - maxBackoff = 60 * time.Second + minBackoff = 100 * time.Millisecond + maxBackoff = 60 * time.Second + jitterPercentage = 0.2 ) -// Next +// Next returns the delay to wait before next retry func (b *Backoff) Next() time.Duration { // Double the delay each time b.backoff += b.backoff @@ -40,6 +47,7 @@ func (b *Backoff) Next() time.Duration { } else if b.backoff.Nanoseconds() > maxBackoff.Nanoseconds() { b.backoff = maxBackoff } + jitter := rand.Float64() * float64(b.backoff) * jitterPercentage - return b.backoff + return b.backoff + time.Duration(jitter) } diff --git a/pulsar/internal/backoff_test.go b/pulsar/internal/backoff_test.go new file mode 100644 index 0000000000..46c92116e8 --- /dev/null +++ b/pulsar/internal/backoff_test.go @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package internal + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestBackoff_NextMinValue(t *testing.T) { + backoff := &Backoff{} + delay := backoff.Next() + assert.GreaterOrEqual(t, int64(delay), int64(100*time.Millisecond)) + assert.LessOrEqual(t, int64(delay), int64(120*time.Millisecond)) +} + +func TestBackoff_NextExponentialBackoff(t *testing.T) { + backoff := &Backoff{} + previousDelay := backoff.Next() + // the last value before capping to the max value is 51.2 s (.1, .2, .4, .8, 1.6, 3.2, 6.4, 12.8, 25.6, 51.2) + for previousDelay < 51*time.Second { + delay := backoff.Next() + // the jitter introduces at most 20% difference so at least delay is 1.6=(1-0.2)*2 bigger + assert.GreaterOrEqual(t, int64(delay), int64(1.6*float64(previousDelay))) + // the jitter introduces at most 20% difference so delay is less than twice the previous value + assert.LessOrEqual(t, int64(float64(delay)*.8), int64(2*float64(previousDelay))) + previousDelay = delay + } +} + +func TestBackoff_NextMaxValue(t *testing.T) { + backoff := &Backoff{} + var delay time.Duration + for delay < maxBackoff { + delay = backoff.Next() + } + + cappedDelay := backoff.Next() + assert.GreaterOrEqual(t, int64(cappedDelay), int64(maxBackoff)) + // max value is 60 seconds + 20% jitter = 72 seconds + assert.LessOrEqual(t, int64(cappedDelay), int64(72*time.Second)) +} From 8139a2c721d0d9f66727fac16f3705bc254d9017 Mon Sep 17 00:00:00 2001 From: dferstay Date: Wed, 30 Jun 2021 06:49:46 -0700 Subject: [PATCH 15/25] Fix panic() in internal/connection when writing to a closed channel during close (#539) The race is as follows: T1 - calls SendRequestNoWait(), checks the connection state, and prepares to enter the select statement T2 - calls TriggerClose() closes cnx and the closeCh T3 - run() go-routine for processing incomingRequestsCh drops into case <-closeCh: and calls failLeftRequestsWhenClose() which drains and closes incomingRequestsCh T1 - resumes and drops into the select where both closeCh and incomingRequestsCh are closed. When two cases of a `select` are valid, the case executed is chosen at random; see https://tour.golang.org/concurrency/5 This commit introduces a connectionClosing state and a wait group to track writes by the SendRequest() methods. * TriggerClose() moves the connection into the connectionClosing state before the closeCh is closed. * The failLeftRequestsWhenClosed() method waits on the waitgroup for outstanding SendRequest() methods to complete before it closes the incomingRequestsCh * The SendRequest() methods first add to the waitgroup before checking the connection state; if the state is either closing or closed, SendRequest() returns an error. With the above it is not possible for thread to attempt to add a request to the incomingRequestsCh without being tracked by the waitgroup, and the incomingRequestsCh will not be closed until operations tracked by the waitgroup have completed. Signed-off-by: Daniel Ferstay Co-authored-by: Daniel Ferstay Co-authored-by: xiaolongran --- pulsar/internal/connection.go | 27 ++++++++++++++++++++++++--- 1 file changed, 24 insertions(+), 3 deletions(-) diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 1e3ce0264b..70c1468147 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -89,6 +89,7 @@ type connectionState int32 const ( connectionInit = iota connectionReady + connectionClosing connectionClosed ) @@ -98,6 +99,8 @@ func (s connectionState) String() string { return "Initializing" case connectionReady: return "Ready" + case connectionClosing: + return "Closing" case connectionClosed: return "Closed" default: @@ -142,6 +145,7 @@ type connection struct { requestIDGenerator uint64 + incomingRequestsWG sync.WaitGroup incomingRequestsCh chan *request incomingCmdCh chan *incomingCmd closeCh chan interface{} @@ -333,10 +337,15 @@ func (c *connection) waitUntilReady() error { } func (c *connection) failLeftRequestsWhenClose() { + // wait for outstanding incoming requests to complete before draining + // and closing the channel + c.incomingRequestsWG.Wait() + reqLen := len(c.incomingRequestsCh) for i := 0; i < reqLen; i++ { c.internalSendRequest(<-c.incomingRequestsCh) } + close(c.incomingRequestsCh) } @@ -549,8 +558,13 @@ func (c *connection) Write(data Buffer) { func (c *connection) SendRequest(requestID uint64, req *pb.BaseCommand, callback func(command *pb.BaseCommand, err error)) { - if c.getState() == connectionClosed { + c.incomingRequestsWG.Add(1) + defer c.incomingRequestsWG.Done() + + state := c.getState() + if state == connectionClosed || state == connectionClosing { callback(req, ErrConnectionClosed) + } else { select { case <-c.closeCh: @@ -566,7 +580,11 @@ func (c *connection) SendRequest(requestID uint64, req *pb.BaseCommand, } func (c *connection) SendRequestNoWait(req *pb.BaseCommand) error { - if c.getState() == connectionClosed { + c.incomingRequestsWG.Add(1) + defer c.incomingRequestsWG.Done() + + state := c.getState() + if state == connectionClosed || state == connectionClosing { return ErrConnectionClosed } @@ -760,6 +778,8 @@ func (c *connection) UnregisterListener(id uint64) { // broadcasting the notification on the close channel func (c *connection) TriggerClose() { c.closeOnce.Do(func() { + c.setState(connectionClosing) + cnx := c.cnx if cnx != nil { cnx.Close() @@ -780,9 +800,10 @@ func (c *connection) Close() { } c.log.Info("Connection closed") + c.TriggerClose() // do not use changeState() since they share the same lock c.setState(connectionClosed) - c.TriggerClose() + c.pingTicker.Stop() c.pingCheckTicker.Stop() From 3e091aec30473f06322d41d374689686b3e56cc1 Mon Sep 17 00:00:00 2001 From: cckellogg Date: Thu, 1 Jul 2021 09:43:00 -0700 Subject: [PATCH 16/25] Simplify connection close logic (#559) ### Motivation Simplify the connection close logic into one function. I thinks this makes the code easier to reason about and maintain. --- pulsar/internal/connection.go | 124 +++++++++++++++------------ pulsar/internal/connection_reader.go | 22 +++-- pulsar/internal/rpc_client.go | 52 +++++------ 3 files changed, 105 insertions(+), 93 deletions(-) diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 70c1468147..2549e21991 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -52,6 +52,10 @@ type TLSOptions struct { ValidateHostname bool } +var ( + errConnectionClosed = errors.New("connection closed") +) + // ConnectionListener is a user of a connection (eg. a producer or // a consumer) that can register itself to get notified // when the connection is closed. @@ -221,11 +225,11 @@ func (c *connection) start() { c.run() } else { c.metrics.ConnectionsHandshakeErrors.Inc() - c.changeState(connectionClosed) + c.Close() } } else { c.metrics.ConnectionsEstablishmentErrors.Inc() - c.changeState(connectionClosed) + c.Close() } }() } @@ -325,7 +329,7 @@ func (c *connection) waitUntilReady() error { defer c.Unlock() for c.getState() != connectionReady { - c.log.Debugf("Wait until connection is ready. State: %s", c.getState().String()) + c.log.Debugf("Wait until connection is ready state=%s", c.getState().String()) if c.getState() == connectionClosed { return errors.New("connection error") } @@ -362,7 +366,7 @@ func (c *connection) run() { // including the final cleanup, to avoid the issue https://github.com/apache/pulsar-client-go/issues/239 c.pendingLock.Lock() for id, req := range c.pendingReqs { - req.callback(nil, errors.New("connection closed")) + req.callback(nil, errConnectionClosed) delete(c.pendingReqs, id) } c.pendingLock.Unlock() @@ -392,7 +396,6 @@ func (c *connection) run() { case cmd := <-c.incomingCmdCh: c.internalReceivedCommand(cmd.cmd, cmd.headersAndPayload) - case data := <-c.writeRequestsCh: if data == nil { return @@ -415,7 +418,7 @@ func (c *connection) runPingCheck() { // We have not received a response to the previous Ping request, the // connection to broker is stale c.log.Warn("Detected stale connection to broker") - c.TriggerClose() + c.Close() return } } @@ -457,7 +460,7 @@ func (c *connection) internalWriteData(data Buffer) { c.log.Debug("Write data: ", data.ReadableBytes()) if _, err := c.cnx.Write(data.ReadableSlice()); err != nil { c.log.WithError(err).Warn("Failed to write on connection") - c.TriggerClose() + c.Close() } } @@ -548,7 +551,7 @@ func (c *connection) internalReceivedCommand(cmd *pb.BaseCommand, headersAndPayl default: c.log.Errorf("Received invalid command type: %s", cmd.Type) - c.TriggerClose() + c.Close() } } @@ -602,7 +605,7 @@ func (c *connection) SendRequestNoWait(req *pb.BaseCommand) error { } func (c *connection) internalSendRequest(req *request) { - if c.getState() == connectionClosed { + if c.closed() { c.log.Warnf("internalSendRequest failed for connectionClosed") if req.callback != nil { req.callback(req.cmd, ErrConnectionClosed) @@ -659,7 +662,9 @@ func (c *connection) handleSendReceipt(response *pb.CommandSendReceipt) { if ok { producer.ReceivedSendReceipt(response) } else { - c.log.WithField("producerID", producerID).Warn("Got unexpected send receipt for message: ", response.MessageId) + c.log. + WithField("producerID", producerID). + Warn("Got unexpected send receipt for messageID=%+v", response.MessageId) } } @@ -713,7 +718,7 @@ func (c *connection) handleAuthChallenge(authChallenge *pb.CommandAuthChallenge) authData, err := c.auth.GetData() if err != nil { c.log.WithError(err).Warn("Failed to load auth credentials") - c.TriggerClose() + c.Close() return } @@ -761,6 +766,12 @@ func (c *connection) handleCloseProducer(closeProducer *pb.CommandCloseProducer) } func (c *connection) RegisterListener(id uint64, listener ConnectionListener) { + // do not add if connection is closed + if c.closed() { + c.log.Warnf("Connection closed unable register listener id=%+v", id) + return + } + c.listenersLock.Lock() defer c.listenersLock.Unlock() @@ -774,62 +785,55 @@ func (c *connection) UnregisterListener(id uint64) { delete(c.listeners, id) } -// TriggerClose the connection close by forcing the socket to close and -// broadcasting the notification on the close channel -func (c *connection) TriggerClose() { +// Close closes the connection by +// closing underlying socket connection and closeCh. +// This also triggers callbacks to the ConnectionClosed listeners. +func (c *connection) Close() { c.closeOnce.Do(func() { - c.setState(connectionClosing) - + c.Lock() cnx := c.cnx + // do not use changeState() since they share the same lock + c.setState(connectionClosed) + c.cond.Broadcast() + c.Unlock() + if cnx != nil { - cnx.Close() + _ = cnx.Close() } close(c.closeCh) - }) -} - -func (c *connection) Close() { - c.Lock() - defer c.Unlock() - c.cond.Broadcast() + c.pingTicker.Stop() + c.pingCheckTicker.Stop() - if c.getState() == connectionClosed { - return - } - - c.log.Info("Connection closed") - c.TriggerClose() - // do not use changeState() since they share the same lock - c.setState(connectionClosed) - - c.pingTicker.Stop() - c.pingCheckTicker.Stop() - - listeners := make(map[uint64]ConnectionListener) - c.listenersLock.RLock() - for id, listener := range c.listeners { - listeners[id] = listener - } - c.listenersLock.RUnlock() + listeners := make(map[uint64]ConnectionListener) + c.listenersLock.Lock() + for id, listener := range c.listeners { + listeners[id] = listener + delete(c.listeners, id) + } + c.listenersLock.Unlock() - for _, listener := range listeners { - listener.ConnectionClosed() - } + consumerHandlers := make(map[uint64]ConsumerHandler) + c.consumerHandlersLock.Lock() + for id, handler := range c.consumerHandlers { + consumerHandlers[id] = handler + delete(c.consumerHandlers, id) + } + c.consumerHandlersLock.Unlock() - consumerHandlers := make(map[uint64]ConsumerHandler) - c.consumerHandlersLock.RLock() - for id, handler := range c.consumerHandlers { - consumerHandlers[id] = handler - } - c.consumerHandlersLock.RUnlock() + // notify producers connection closed + for _, listener := range listeners { + listener.ConnectionClosed() + } - for _, handler := range consumerHandlers { - handler.ConnectionClosed() - } + // notify consumers connection closed + for _, handler := range consumerHandlers { + handler.ConnectionClosed() + } - c.metrics.ConnectionsClosed.Inc() + c.metrics.ConnectionsClosed.Inc() + }) } func (c *connection) changeState(state connectionState) { @@ -847,6 +851,10 @@ func (c *connection) setState(state connectionState) { c.state.Store(int32(state)) } +func (c *connection) closed() bool { + return connectionClosed == c.getState() +} + func (c *connection) newRequestID() uint64 { return atomic.AddUint64(&c.requestIDGenerator, 1) } @@ -886,6 +894,12 @@ func (c *connection) getTLSConfig() (*tls.Config, error) { } func (c *connection) AddConsumeHandler(id uint64, handler ConsumerHandler) { + // do not add if connection is closed + if c.closed() { + c.log.Warnf("Closed connection unable add consumer with id=%+v", id) + return + } + c.consumerHandlersLock.Lock() defer c.consumerHandlersLock.Unlock() c.consumerHandlers[id] = handler diff --git a/pulsar/internal/connection_reader.go b/pulsar/internal/connection_reader.go index 62a65266ce..97078c578d 100644 --- a/pulsar/internal/connection_reader.go +++ b/pulsar/internal/connection_reader.go @@ -24,7 +24,6 @@ import ( pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" "github.com/gogo/protobuf/proto" - "github.com/pkg/errors" ) type connectionReader struct { @@ -45,8 +44,10 @@ func (r *connectionReader) readFromConnection() { for { cmd, headersAndPayload, err := r.readSingleCommand() if err != nil { - r.cnx.log.WithError(err).Info("Error reading from connection") - r.cnx.TriggerClose() + if !r.cnx.closed() { + r.cnx.log.WithError(err).Infof("Error reading from connection") + r.cnx.Close() + } break } @@ -68,7 +69,7 @@ func (r *connectionReader) readSingleCommand() (cmd *pb.BaseCommand, headersAndP r.buffer.Clear() } if err := r.readAtLeast(4); err != nil { - return nil, nil, errors.Errorf("Short read when reading frame size: %s", err) + return nil, nil, fmt.Errorf("unable to read frame size: %+v", err) } } @@ -78,7 +79,7 @@ func (r *connectionReader) readSingleCommand() (cmd *pb.BaseCommand, headersAndP if r.cnx.maxMessageSize != 0 && int32(frameSize) > maxFrameSize { frameSizeError := fmt.Errorf("received too big frame size=%d maxFrameSize=%d", frameSize, maxFrameSize) r.cnx.log.Error(frameSizeError) - r.cnx.TriggerClose() + r.cnx.Close() return nil, nil, frameSizeError } @@ -86,7 +87,8 @@ func (r *connectionReader) readSingleCommand() (cmd *pb.BaseCommand, headersAndP if r.buffer.ReadableBytes() < frameSize { remainingBytes := frameSize - r.buffer.ReadableBytes() if err := r.readAtLeast(remainingBytes); err != nil { - return nil, nil, errors.Errorf("Short read when reading frame: %s", err) + return nil, nil, + fmt.Errorf("unable to read frame: %+v", err) } } @@ -122,7 +124,11 @@ func (r *connectionReader) readAtLeast(size uint32) error { n, err := io.ReadAtLeast(r.cnx.cnx, r.buffer.WritableSlice(), int(size)) if err != nil { - r.cnx.TriggerClose() + // has the connection been closed? + if r.cnx.closed() { + return errConnectionClosed + } + r.cnx.Close() return err } @@ -135,7 +141,7 @@ func (r *connectionReader) deserializeCmd(data []byte) (*pb.BaseCommand, error) err := proto.Unmarshal(data, cmd) if err != nil { r.cnx.log.WithError(err).Warn("Failed to parse protobuf command") - r.cnx.TriggerClose() + r.cnx.Close() return nil, err } return cmd, nil diff --git a/pulsar/internal/rpc_client.go b/pulsar/internal/rpc_client.go index 1374747ab8..24506ef06b 100644 --- a/pulsar/internal/rpc_client.go +++ b/pulsar/internal/rpc_client.go @@ -19,7 +19,6 @@ package internal import ( "errors" - "net" "net/url" "sync/atomic" "time" @@ -88,37 +87,30 @@ func NewRPCClient(serviceURL *url.URL, serviceNameResolver ServiceNameResolver, func (c *rpcClient) RequestToAnyBroker(requestID uint64, cmdType pb.BaseCommand_Type, message proto.Message) (*RPCResult, error) { - host, err := c.serviceNameResolver.ResolveHost() - if err != nil { - c.log.Errorf("request host resolve failed with error: {%v}", err) - return nil, err - } - rpcResult, err := c.Request(host, host, requestID, cmdType, message) - if _, ok := err.(net.Error); ok || (err != nil && err.Error() == "connection error") { - // We can retry this kind of requests over a connection error because they're - // not specific to a particular broker. - backoff := Backoff{100 * time.Millisecond} - startTime := time.Now() - var retryTime time.Duration - - for time.Since(startTime) < c.requestTimeout { - retryTime = backoff.Next() - c.log.Debugf("Retrying request in {%v} with timeout in {%v}", retryTime, c.requestTimeout) - time.Sleep(retryTime) - host, err = c.serviceNameResolver.ResolveHost() - if err != nil { - c.log.Errorf("Retrying request host resolve failed with error: {%v}", err) - continue - } - rpcResult, err = c.Request(host, host, requestID, cmdType, message) - if _, ok := err.(net.Error); ok || (err != nil && err.Error() == "connection error") { - continue - } else { - // We either succeeded or encountered a non connection error - break - } + var err error + var host *url.URL + var rpcResult *RPCResult + startTime := time.Now() + backoff := Backoff{100 * time.Millisecond} + // we can retry these requests because this kind of request is + // not specific to any particular broker + for time.Since(startTime) < c.requestTimeout { + host, err = c.serviceNameResolver.ResolveHost() + if err != nil { + c.log.WithError(err).Errorf("rpc client failed to resolve host") + return nil, err } + rpcResult, err = c.Request(host, host, requestID, cmdType, message) + // success we got a response + if err == nil { + break + } + + retryTime := backoff.Next() + c.log.Debugf("Retrying request in {%v} with timeout in {%v}", retryTime, c.requestTimeout) + time.Sleep(retryTime) } + return rpcResult, err } From 7a60a7e529d6a52d52a1a5180dfaedebb8eb6987 Mon Sep 17 00:00:00 2001 From: Prabhudas Date: Mon, 5 Jul 2021 12:56:36 +0530 Subject: [PATCH 17/25] add base package for encryption (#555) Breakdown of PR [#552 ](https://github.com/apache/pulsar-client-go/pull/552) This PR includes base interface and default implementation. --- go.sum | 1 + pulsar/crypto/crypto_failure_action.go | 43 +++ pulsar/crypto/crypto_key_reader.go | 28 ++ pulsar/crypto/default_crypto_Key_reader.go | 51 +++ .../crypto/default_crypto_key_reader_test.go | 62 +++ pulsar/crypto/default_message_crypto.go | 365 ++++++++++++++++++ pulsar/crypto/default_message_crypto_test.go | 165 ++++++++ pulsar/crypto/encryption_key_Info.go | 49 +++ pulsar/crypto/message_crypto.go | 35 ++ pulsar/crypto/message_metadata.go | 122 ++++++ pulsar/crypto/message_metadata_test.go | 117 ++++++ pulsar/crypto/testdata/pri_key_rsa.pem | 39 ++ pulsar/crypto/testdata/pub_key_rsa.pem | 11 + .../crypto/testdata/truncated_pub_key_rsa.pem | 10 + .../testdata/wrong_encode_pub_key_rsa.pem | 9 + .../testdata/wrong_encoded_pri_key_rsa.pem | 38 ++ 16 files changed, 1145 insertions(+) create mode 100644 pulsar/crypto/crypto_failure_action.go create mode 100644 pulsar/crypto/crypto_key_reader.go create mode 100644 pulsar/crypto/default_crypto_Key_reader.go create mode 100644 pulsar/crypto/default_crypto_key_reader_test.go create mode 100644 pulsar/crypto/default_message_crypto.go create mode 100644 pulsar/crypto/default_message_crypto_test.go create mode 100644 pulsar/crypto/encryption_key_Info.go create mode 100644 pulsar/crypto/message_crypto.go create mode 100644 pulsar/crypto/message_metadata.go create mode 100644 pulsar/crypto/message_metadata_test.go create mode 100644 pulsar/crypto/testdata/pri_key_rsa.pem create mode 100644 pulsar/crypto/testdata/pub_key_rsa.pem create mode 100644 pulsar/crypto/testdata/truncated_pub_key_rsa.pem create mode 100644 pulsar/crypto/testdata/wrong_encode_pub_key_rsa.pem create mode 100644 pulsar/crypto/testdata/wrong_encoded_pri_key_rsa.pem diff --git a/go.sum b/go.sum index c56f5b7319..b2818eec4e 100644 --- a/go.sum +++ b/go.sum @@ -165,6 +165,7 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.5.1 h1:nOGnQDM7FYENwehXlg/kFVnos3rEvtKTjRvOWSzb6H4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= diff --git a/pulsar/crypto/crypto_failure_action.go b/pulsar/crypto/crypto_failure_action.go new file mode 100644 index 0000000000..891d7405d7 --- /dev/null +++ b/pulsar/crypto/crypto_failure_action.go @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package crypto + +const ( + // ProducerCryptoFailureActionFail this is the default option to fail send if crypto operation fails. + ProducerCryptoFailureActionFail = iota + + // ProducerCryptoFailureActionSend ingnore crypto failure and proceed with sending unencrypted message. + ProducerCryptoFailureActionSend +) + +const ( + // ConsumerCryptoFailureActionFail this is the default option to fail consume messages until crypto succeeds. + ConsumerCryptoFailureActionFail = iota + + // ConsumerCryptoFailureActionDiscard message is silently acknowledged and not delivered to the application + ConsumerCryptoFailureActionDiscard + + // ConsumerCryptoFailureActionConsume deliver the encrypted message to the application. + // It's the application's responsibility to decrypt the message. + // if message is also compressed, decompression will fail. + // If message contain batch messages, client will not be able to retrieve + // individual messages in the batch. + // delivered encrypted message contains EncryptionContext which contains encryption + // and compression information in it using which application can decrypt the payload. + ConsumerCryptoFailureActionConsume +) diff --git a/pulsar/crypto/crypto_key_reader.go b/pulsar/crypto/crypto_key_reader.go new file mode 100644 index 0000000000..51295c1300 --- /dev/null +++ b/pulsar/crypto/crypto_key_reader.go @@ -0,0 +1,28 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package crypto + +// KeyReader implement this interface to read and provide public & private keys +// key pair can be RSA, ECDSA +type KeyReader interface { + // PublicKey get public key that is be used by the producer to encrypt data key + PublicKey(keyName string, metadata map[string]string) (*EncryptionKeyInfo, error) + + // PrivateKey get private key that is used by the consumer to decrypt data key + PrivateKey(keyName string, metadata map[string]string) (*EncryptionKeyInfo, error) +} diff --git a/pulsar/crypto/default_crypto_Key_reader.go b/pulsar/crypto/default_crypto_Key_reader.go new file mode 100644 index 0000000000..6378d4e9ae --- /dev/null +++ b/pulsar/crypto/default_crypto_Key_reader.go @@ -0,0 +1,51 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package crypto + +import "io/ioutil" + +// FileKeyReader default implementation of KeyReader +type FileKeyReader struct { + publicKeyPath string + privateKeyPath string +} + +func NewFileKeyReader(publicKeyPath, privateKeyPath string) *FileKeyReader { + return &FileKeyReader{ + publicKeyPath: publicKeyPath, + privateKeyPath: privateKeyPath, + } +} + +// PublicKey read public key from the given path +func (d *FileKeyReader) PublicKey(keyName string, keyMeta map[string]string) (*EncryptionKeyInfo, error) { + return readKey(keyName, d.publicKeyPath, keyMeta) +} + +// PrivateKey read private key from the given path +func (d *FileKeyReader) PrivateKey(keyName string, keyMeta map[string]string) (*EncryptionKeyInfo, error) { + return readKey(keyName, d.privateKeyPath, keyMeta) +} + +func readKey(keyName, path string, keyMeta map[string]string) (*EncryptionKeyInfo, error) { + key, err := ioutil.ReadFile(path) + if err != nil { + return nil, err + } + return NewEncryptionKeyInfo(keyName, key, keyMeta), nil +} diff --git a/pulsar/crypto/default_crypto_key_reader_test.go b/pulsar/crypto/default_crypto_key_reader_test.go new file mode 100644 index 0000000000..776d08fabe --- /dev/null +++ b/pulsar/crypto/default_crypto_key_reader_test.go @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package crypto + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestGetPublicKey(t *testing.T) { + keyReader := NewFileKeyReader("../crypto/testdata/pub_key_rsa.pem", "") + keyInfo, err := keyReader.PublicKey("test-key", map[string]string{"key": "value"}) + + assert.Nil(t, err) + assert.NotNil(t, keyInfo) + assert.NotEmpty(t, keyInfo.Metadata()) + assert.NotEmpty(t, keyInfo.Name()) + assert.NotEmpty(t, keyInfo.Key()) + assert.Equal(t, "value", keyInfo.metadata["key"]) +} + +func TestGetPrivateKey(t *testing.T) { + keyReader := NewFileKeyReader("", "../crypto/testdata/pri_key_rsa.pem") + keyInfo, err := keyReader.PrivateKey("test-key", map[string]string{"key": "value"}) + + assert.Nil(t, err) + assert.NotNil(t, keyInfo) + assert.NotEmpty(t, keyInfo.Metadata()) + assert.NotEmpty(t, keyInfo.Name()) + assert.NotEmpty(t, keyInfo.Key()) + assert.Equal(t, "value", keyInfo.metadata["key"]) +} + +func TestInvalidKeyPath(t *testing.T) { + keyReader := NewFileKeyReader("../crypto/testdata/no_pub_key_rsa.pem", "../crypto/testdata/no_pri_key_rsa.pem") + + // try to read public key + keyInfo, err := keyReader.PublicKey("test-pub-key", nil) + assert.Nil(t, keyInfo) + assert.NotNil(t, err) + + // try to read private key + keyInfo, err = keyReader.PrivateKey("test-pri-key", nil) + assert.Nil(t, keyInfo) + assert.NotNil(t, err) +} diff --git a/pulsar/crypto/default_message_crypto.go b/pulsar/crypto/default_message_crypto.go new file mode 100644 index 0000000000..93a024448d --- /dev/null +++ b/pulsar/crypto/default_message_crypto.go @@ -0,0 +1,365 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package crypto + +import ( + gocrypto "crypto" + "crypto/aes" + "crypto/cipher" + "crypto/md5" + "crypto/rand" + "crypto/rsa" + "crypto/sha1" + "crypto/x509" + "encoding/pem" + "errors" + "fmt" + "sync" + + "github.com/apache/pulsar-client-go/pulsar/log" +) + +// DefaultMessageCrypto implmentation of the interface MessageCryto +type DefaultMessageCrypto struct { + // data key which is used to encrypt/decrypt messages + dataKey []byte + + // LoadingCache used by the consumer to cache already decrypted key + loadingCache sync.Map // map[string][]byte + + encryptedDataKeyMap sync.Map // map[string]EncryptionKeyInfo + + logCtx string + + logger log.Logger + + cipherLock sync.Mutex + + encryptLock sync.Mutex +} + +// NewDefaultMessageCrypto get the instance of message crypto +func NewDefaultMessageCrypto(logCtx string, keyGenNeeded bool, logger log.Logger) (*DefaultMessageCrypto, error) { + + d := &DefaultMessageCrypto{ + logCtx: logCtx, + loadingCache: sync.Map{}, + encryptedDataKeyMap: sync.Map{}, + logger: logger, + } + + if keyGenNeeded { + key, err := generateDataKey() + if err != nil { + return d, err + } + d.dataKey = key + } + + return d, nil +} + +// AddPublicKeyCipher encrypt data key using keyCrypto and cache +func (d *DefaultMessageCrypto) AddPublicKeyCipher(keyNames []string, keyReader KeyReader) error { + key, err := generateDataKey() + if err != nil { + return err + } + + d.dataKey = key + for _, keyName := range keyNames { + err := d.addPublicKeyCipher(keyName, keyReader) + if err != nil { + return err + } + } + return nil +} + +func (d *DefaultMessageCrypto) addPublicKeyCipher(keyName string, keyReader KeyReader) error { + d.cipherLock.Lock() + defer d.cipherLock.Unlock() + if keyName == "" || keyReader == nil { + return fmt.Errorf("keyname or keyreader is null") + } + + // read the public key and its info using keyReader + keyInfo, err := keyReader.PublicKey(keyName, nil) + if err != nil { + return err + } + + parsedKey, err := d.loadPublicKey(keyInfo.Key()) + if err != nil { + return err + } + + // try to cast to RSA key + rsaPubKey, ok := parsedKey.(*rsa.PublicKey) + if !ok { + return fmt.Errorf("only RSA keys are supported") + } + + encryptedDataKey, err := rsa.EncryptOAEP(sha1.New(), rand.Reader, rsaPubKey, d.dataKey, nil) + if err != nil { + return err + } + + d.encryptedDataKeyMap.Store(keyName, NewEncryptionKeyInfo(keyName, encryptedDataKey, keyInfo.Metadata())) + + return nil +} + +// RemoveKeyCipher remove encrypted data key from cache +func (d *DefaultMessageCrypto) RemoveKeyCipher(keyName string) bool { + if keyName == "" { + return false + } + d.encryptedDataKeyMap.Delete(keyName) + return true +} + +// Encrypt encrypt payload using encryption keys and add encrypted data key +// to message metadata. Here data key is encrypted +// using public key +func (d *DefaultMessageCrypto) Encrypt(encKeys []string, + keyReader KeyReader, + msgMetadata MessageMetadataSupplier, + payload []byte) ([]byte, error) { + d.encryptLock.Lock() + defer d.encryptLock.Unlock() + + if len(encKeys) == 0 { + return payload, nil + } + + for _, keyName := range encKeys { + // if key is not already loaded, load it + if _, ok := d.encryptedDataKeyMap.Load(keyName); !ok { + if err := d.addPublicKeyCipher(keyName, keyReader); err != nil { + d.logger.Error(err) + } + } + + // add key to the message metadata + if k, ok := d.encryptedDataKeyMap.Load(keyName); ok { + keyInfo, keyInfoOk := k.(*EncryptionKeyInfo) + + if keyInfoOk { + msgMetadata.UpsertEncryptionkey(*keyInfo) + } else { + d.logger.Error("failed to get EncryptionKeyInfo for key %v", keyName) + } + } else { + // we should never reach here + msg := fmt.Sprintf("%v Failed to find encrypted Data key for key %v", d.logCtx, keyName) + d.logger.Errorf(msg) + return nil, fmt.Errorf(msg) + } + + } + + // generate a new AES cipher with data key + c, err := aes.NewCipher(d.dataKey) + + if err != nil { + d.logger.Error("failed to create AES cipher") + return nil, err + } + + // gcm + gcm, err := cipher.NewGCM(c) + + if err != nil { + d.logger.Error("failed to create gcm") + return nil, err + } + + // create gcm param + nonce := make([]byte, gcm.NonceSize()) + _, err = rand.Read(nonce) + + if err != nil { + d.logger.Error(err) + return nil, err + } + + // Update message metadata with encryption param + msgMetadata.SetEncryptionParam(nonce) + + // encrypt payload using seal function + return gcm.Seal(nil, nonce, payload, nil), nil +} + +// Decrypt decrypt the payload using decrypted data key. +// Here data key is read from from the message +// metadata and decrypted using private key. +func (d *DefaultMessageCrypto) Decrypt(msgMetadata MessageMetadataSupplier, + payload []byte, + keyReader KeyReader) ([]byte, error) { + // if data key is present, attempt to derypt using the existing key + if d.dataKey != nil { + decryptedData, err := d.getKeyAndDecryptData(msgMetadata, payload) + if err != nil { + d.logger.Error(err) + } + + if decryptedData != nil { + return decryptedData, nil + } + } + + // data key is null or decryption failed. Attempt to regenerate data key + encKeys := msgMetadata.EncryptionKeys() + var ecKeyInfo *EncryptionKeyInfo + + for _, encKey := range encKeys { + if d.decryptDataKey(encKey.Name(), encKey.Key(), encKey.Metadata(), keyReader) { + ecKeyInfo = &encKey + } + } + + if ecKeyInfo == nil || d.dataKey == nil { + // unable to decrypt data key + return nil, errors.New("unable to decrypt data key") + } + + return d.getKeyAndDecryptData(msgMetadata, payload) +} + +func (d *DefaultMessageCrypto) decryptData(dataKeySecret []byte, + msgMetadata MessageMetadataSupplier, + payload []byte) ([]byte, error) { + // get nonce from message metadata + nonce := msgMetadata.EncryptionParam() + + c, err := aes.NewCipher(dataKeySecret) + + if err != nil { + d.logger.Error(err) + return nil, err + } + + gcm, err := cipher.NewGCM(c) + if err != nil { + return nil, err + } + + decryptedData, err := gcm.Open(nil, nonce, payload, nil) + + if err != nil { + d.logger.Error(err) + } + + return decryptedData, err +} + +func (d *DefaultMessageCrypto) getKeyAndDecryptData(msgMetadata MessageMetadataSupplier, + payload []byte) ([]byte, error) { + // go through all keys to retrieve data key from cache + for _, k := range msgMetadata.EncryptionKeys() { + msgDataKey := k.Key() + keyDigest := fmt.Sprintf("%x", md5.Sum(msgDataKey)) + if storedSecretKey, ok := d.loadingCache.Load(keyDigest); ok { + decryptedData, err := d.decryptData(storedSecretKey.([]byte), msgMetadata, payload) + if err != nil { + d.logger.Error(err) + } + + if decryptedData != nil { + return decryptedData, nil + } + } else { + // First time, entry wont be present in cache + d.logger.Debugf("%s Failed to decrypt data or data key is not in cache. Will attempt to refresh", d.logCtx) + } + } + return nil, nil +} + +func (d *DefaultMessageCrypto) decryptDataKey(keyName string, + encDatakey []byte, + keyMeta map[string]string, + keyReader KeyReader) bool { + + keyInfo, err := keyReader.PrivateKey(keyName, keyMeta) + if err != nil { + d.logger.Error(err) + return false + } + + parsedKey, err := d.loadPrivateKey(keyInfo.Key()) + if err != nil { + d.logger.Error(err) + return false + } + + rsaPriKey, ok := parsedKey.(*rsa.PrivateKey) + if !ok { + d.logger.Error("only RSA keys are supported") + return false + } + + decryptedDataKey, err := rsa.DecryptOAEP(sha1.New(), rand.Reader, rsaPriKey, encDatakey, nil) + if err != nil { + d.logger.Error(err) + return false + } + d.dataKey = decryptedDataKey + d.loadingCache.Store(fmt.Sprintf("%x", md5.Sum(encDatakey)), d.dataKey) + + return true +} + +func (d *DefaultMessageCrypto) loadPrivateKey(key []byte) (gocrypto.PrivateKey, error) { + var privateKey gocrypto.PrivateKey + priPem, _ := pem.Decode(key) + if priPem == nil { + return privateKey, fmt.Errorf("failed to decode private key") + } + genericPrivateKey, err := x509.ParsePKCS1PrivateKey(priPem.Bytes) + if err != nil { + return privateKey, err + } + privateKey = genericPrivateKey + return privateKey, nil +} + +// read the public key into RSA key +func (d *DefaultMessageCrypto) loadPublicKey(key []byte) (gocrypto.PublicKey, error) { + var publickKey gocrypto.PublicKey + + pubPem, _ := pem.Decode(key) + if pubPem == nil { + return publickKey, fmt.Errorf("failed to decode public key") + } + + genericPublicKey, err := x509.ParsePKIXPublicKey(pubPem.Bytes) + if err != nil { + return publickKey, err + } + publickKey = genericPublicKey + + return publickKey, nil +} + +func generateDataKey() ([]byte, error) { + key := make([]byte, 32) // generate key of length 256 bits + _, err := rand.Read(key) // cryptographically secure random number + return key, err +} diff --git a/pulsar/crypto/default_message_crypto_test.go b/pulsar/crypto/default_message_crypto_test.go new file mode 100644 index 0000000000..ae126e0a3a --- /dev/null +++ b/pulsar/crypto/default_message_crypto_test.go @@ -0,0 +1,165 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package crypto + +import ( + "testing" + + pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" + "github.com/apache/pulsar-client-go/pulsar/log" + "github.com/stretchr/testify/assert" +) + +func TestAddPublicKeyCipher(t *testing.T) { + msgCrypto, err := NewDefaultMessageCrypto("test-default-crypto", true, log.DefaultNopLogger()) + assert.Nil(t, err) + assert.NotNil(t, msgCrypto) + + // valid keyreader + err = msgCrypto.AddPublicKeyCipher( + []string{"my-app.key"}, + NewFileKeyReader("../crypto/testdata/pub_key_rsa.pem", ""), + ) + assert.Nil(t, err) + + // invalid keyreader + err = msgCrypto.AddPublicKeyCipher( + []string{"my-app0.key"}, + NewFileKeyReader("../crypto/testdata/no_pub_key_rsa.pem", ""), + ) + assert.NotNil(t, err) + + // empty keyreader + err = msgCrypto.AddPublicKeyCipher( + []string{"my-app1.key"}, + nil, + ) + assert.NotNil(t, err) + + // keyreader with wrong econding of public key + err = msgCrypto.AddPublicKeyCipher( + []string{"my-app2.key"}, + NewFileKeyReader("../crypto/testdata/wrong_encode_pub_key_rsa.pem", ""), + ) + assert.NotNil(t, err) + + // keyreader with truncated pub key + err = msgCrypto.AddPublicKeyCipher( + []string{"my-app2.key"}, + NewFileKeyReader("../crypto/testdata/truncated_pub_key_rsa.pem", ""), + ) + assert.NotNil(t, err) +} + +func TestEncrypt(t *testing.T) { + msgMetadata := &pb.MessageMetadata{} + msgMetadataSupplier := NewMessageMetadataSupplier(msgMetadata) + + msg := "my-message-01" + + msgCrypto, err := NewDefaultMessageCrypto("my-app", true, log.DefaultNopLogger()) + assert.Nil(t, err) + assert.NotNil(t, msgCrypto) + + // valid keyreader + encryptedData, err := msgCrypto.Encrypt( + []string{"my-app.key"}, + NewFileKeyReader("../crypto/testdata/pub_key_rsa.pem", ""), + msgMetadataSupplier, + []byte(msg), + ) + + assert.Nil(t, err) + assert.NotNil(t, encryptedData) + + // encrypted data key and encryption param must set in + // in the message metadata after encryption + assert.NotNil(t, msgMetadataSupplier.EncryptionParam()) + assert.NotEmpty(t, msgMetadataSupplier.EncryptionKeys()) + + // invalid keyreader + encryptedData, err = msgCrypto.Encrypt( + []string{"my-app2.key"}, + NewFileKeyReader("../crypto/testdata/no_pub_key_rsa.pem", ""), + msgMetadataSupplier, + []byte(msg), + ) + + assert.NotNil(t, err) + assert.Nil(t, encryptedData) +} + +func TestEncryptDecrypt(t *testing.T) { + msgMetadata := &pb.MessageMetadata{} + msgMetadataSupplier := NewMessageMetadataSupplier(msgMetadata) + + msg := "my-message-01" + + msgCrypto, err := NewDefaultMessageCrypto("my-app", true, log.DefaultNopLogger()) + assert.Nil(t, err) + assert.NotNil(t, msgCrypto) + + // valid keyreader + encryptedData, err := msgCrypto.Encrypt( + []string{"my-app.key"}, + NewFileKeyReader("../crypto/testdata/pub_key_rsa.pem", ""), + msgMetadataSupplier, + []byte(msg), + ) + + assert.Nil(t, err) + assert.NotNil(t, encryptedData) + + // encrypted data key and encryption param must set in + // in the message metadata after encryption + assert.NotNil(t, msgMetadataSupplier.EncryptionParam()) + assert.NotEmpty(t, msgMetadataSupplier.EncryptionKeys()) + + // try to decrypt + msgCryptoDecrypt, err := NewDefaultMessageCrypto("my-app", true, log.DefaultNopLogger()) + assert.Nil(t, err) + assert.NotNil(t, msgCrypto) + + // keyreader with invalid private key + decryptedData, err := msgCryptoDecrypt.Decrypt( + msgMetadataSupplier, + encryptedData, + NewFileKeyReader("", "../crypto/testdata/no_pri_key_rsa.pem"), + ) + assert.NotNil(t, err) + assert.Nil(t, decryptedData) + + // keyreader with wrong encoded private key + decryptedData, err = msgCryptoDecrypt.Decrypt( + msgMetadataSupplier, + encryptedData, + NewFileKeyReader("", "../crypto/testdata/wrong_encoded_pri_key_rsa.pem"), + ) + assert.NotNil(t, err) + assert.Nil(t, decryptedData) + + // keyreader with valid private key + decryptedData, err = msgCryptoDecrypt.Decrypt( + msgMetadataSupplier, + encryptedData, + NewFileKeyReader("", "../crypto/testdata/pri_key_rsa.pem"), + ) + + assert.Nil(t, err) + assert.Equal(t, msg, string(decryptedData)) +} diff --git a/pulsar/crypto/encryption_key_Info.go b/pulsar/crypto/encryption_key_Info.go new file mode 100644 index 0000000000..8418682ae0 --- /dev/null +++ b/pulsar/crypto/encryption_key_Info.go @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package crypto + +// EncryptionKeyInfo +type EncryptionKeyInfo struct { + metadata map[string]string + key []byte + name string +} + +// NewEncryptionKeyInfo +func NewEncryptionKeyInfo(name string, key []byte, metadata map[string]string) *EncryptionKeyInfo { + return &EncryptionKeyInfo{ + metadata: metadata, + name: name, + key: key, + } +} + +// GetKey get key +func (eci *EncryptionKeyInfo) Name() string { + return eci.name +} + +// GetValue get value +func (eci *EncryptionKeyInfo) Key() []byte { + return eci.key +} + +// GetMetadata get key metadata +func (eci *EncryptionKeyInfo) Metadata() map[string]string { + return eci.metadata +} diff --git a/pulsar/crypto/message_crypto.go b/pulsar/crypto/message_crypto.go new file mode 100644 index 0000000000..4ad1eecd88 --- /dev/null +++ b/pulsar/crypto/message_crypto.go @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package crypto + +// MessageCrypto implement this interface to encrypt and decrypt messages +type MessageCrypto interface { + // AddPublicKeyCipher + AddPublicKeyCipher(keyNames []string, keyReader KeyReader) error + + // RemoveKeyCipher remove the key indentified by the keyname from the list + RemoveKeyCipher(keyName string) bool + + // Encrypt the payload using the data key and update + // message metadata with the keyname and encrypted data key + Encrypt(encKeys []string, KeyReader KeyReader, msgMetadata MessageMetadataSupplier, payload []byte) ([]byte, error) + + // Decrypt the payload using the data key. + // Keys used to ecnrypt the data key can be retrieved from msgMetadata + Decrypt(msgMetadata MessageMetadataSupplier, payload []byte, KeyReader KeyReader) ([]byte, error) +} diff --git a/pulsar/crypto/message_metadata.go b/pulsar/crypto/message_metadata.go new file mode 100644 index 0000000000..042a6a0bd0 --- /dev/null +++ b/pulsar/crypto/message_metadata.go @@ -0,0 +1,122 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package crypto + +import ( + pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" +) + +// MessageMetadataSupplier wrapper implementation around message metadata +type MessageMetadataSupplier interface { + // GetEncryptionKeys read all the encryption keys from the MessageMetadata + EncryptionKeys() []EncryptionKeyInfo + + // UpsertEncryptionkey add new or update existing EncryptionKeys in to the MessageMetadata + UpsertEncryptionkey(EncryptionKeyInfo) + + // GetEncryptionParam read the ecryption parameter from the MessageMetadata + EncryptionParam() []byte + + // SetEncryptionParam set encryption parameter in to the MessageMetadata + SetEncryptionParam([]byte) +} + +type MessageMetadata struct { + messageMetadata *pb.MessageMetadata +} + +func NewMessageMetadataSupplier(messageMetadata *pb.MessageMetadata) MessageMetadataSupplier { + return &MessageMetadata{ + messageMetadata: messageMetadata, + } +} + +func (m *MessageMetadata) EncryptionKeys() []EncryptionKeyInfo { + if m.messageMetadata != nil { + encInfo := []EncryptionKeyInfo{} + for _, k := range m.messageMetadata.EncryptionKeys { + key := NewEncryptionKeyInfo(k.GetKey(), k.GetValue(), getKeyMetaMap(k.GetMetadata())) + encInfo = append(encInfo, *key) + } + return encInfo + } + return nil +} + +func (m *MessageMetadata) UpsertEncryptionkey(keyInfo EncryptionKeyInfo) { + if m.messageMetadata != nil { + idx := m.encryptionKeyPresent(keyInfo) + newKey := &pb.EncryptionKeys{ + Key: &keyInfo.name, + Value: keyInfo.Key(), + Metadata: getKeyMeta(keyInfo.Metadata()), + } + + if idx >= 0 { + m.messageMetadata.EncryptionKeys[idx] = newKey + } else { + m.messageMetadata.EncryptionKeys = append(m.messageMetadata.EncryptionKeys, newKey) + } + } +} + +func (m *MessageMetadata) EncryptionParam() []byte { + if m.messageMetadata != nil { + return m.messageMetadata.EncryptionParam + } + return nil +} + +func (m *MessageMetadata) SetEncryptionParam(param []byte) { + if m.messageMetadata != nil { + m.messageMetadata.EncryptionParam = param + } +} + +func (m *MessageMetadata) encryptionKeyPresent(keyInfo EncryptionKeyInfo) int { + if len(m.messageMetadata.EncryptionKeys) > 0 { + for idx, k := range m.messageMetadata.EncryptionKeys { + if k.GetKey() == keyInfo.Name() { + return idx + } + } + } + return -1 +} + +func getKeyMeta(metaMap map[string]string) []*pb.KeyValue { + if len(metaMap) > 0 { + meta := []*pb.KeyValue{} + for k, v := range metaMap { + meta = append(meta, &pb.KeyValue{Key: &k, Value: &v}) + } + return meta + } + return nil +} + +func getKeyMetaMap(keyValues []*pb.KeyValue) map[string]string { + if keyValues != nil { + meta := map[string]string{} + for _, kv := range keyValues { + meta[kv.GetKey()] = kv.GetValue() + } + return meta + } + return nil +} diff --git a/pulsar/crypto/message_metadata_test.go b/pulsar/crypto/message_metadata_test.go new file mode 100644 index 0000000000..3e8cb388b0 --- /dev/null +++ b/pulsar/crypto/message_metadata_test.go @@ -0,0 +1,117 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package crypto + +import ( + "testing" + + pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" + "github.com/stretchr/testify/assert" +) + +func TestGetEncyptionKeys(t *testing.T) { + msgMetadata := &pb.MessageMetadata{} + name1 := "key-1" + value1 := []byte{1, 2, 3, 4} + + name2 := "key-2" + value2 := []byte{4, 3, 2, 1} + + name3 := "key-3" + value3 := []byte{6, 7, 8, 9} + + msgMetadata.EncryptionKeys = append(msgMetadata.EncryptionKeys, &pb.EncryptionKeys{ + Key: &name1, + Value: value1, + Metadata: []*pb.KeyValue{ + {Key: &name1, Value: &name1}, + }, + }, + &pb.EncryptionKeys{ + Key: &name2, + Value: value2, + Metadata: []*pb.KeyValue{ + {Key: &name1, Value: &name1}, + {Key: &name2, Value: &name2}, + }, + }, + &pb.EncryptionKeys{ + Key: &name3, + Value: value3, + }, + ) + + expected := []EncryptionKeyInfo{ + { + name: name1, + key: value1, + metadata: map[string]string{ + "key-1": "key-1", + }, + }, + { + name: name2, + key: value2, + metadata: map[string]string{ + "key-1": "key-1", + "key-2": "key-2", + }, + }, + { + name: name3, + key: value3, + }, + } + + msgMetadataSupplier := NewMessageMetadataSupplier(msgMetadata) + actual := msgMetadataSupplier.EncryptionKeys() + + assert.EqualValues(t, expected, actual) +} + +func TestUpsertEncryptionKey(t *testing.T) { + msgMetadata := &pb.MessageMetadata{} + key1 := "key-1" + value1 := []byte{1, 2, 3, 4} + + keyInfo := NewEncryptionKeyInfo(key1, value1, map[string]string{"key-1": "value-1"}) + + expected := []EncryptionKeyInfo{*keyInfo} + + msgMetadataSupplier := NewMessageMetadataSupplier(msgMetadata) + + msgMetadataSupplier.UpsertEncryptionkey(*keyInfo) + + // try to add same key again + msgMetadataSupplier.UpsertEncryptionkey(*keyInfo) + + actual := msgMetadataSupplier.EncryptionKeys() + + assert.EqualValues(t, expected, actual) +} + +func TestEncryptionParam(t *testing.T) { + msgMetadata := &pb.MessageMetadata{} + + expected := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9} + + msgMetadataSupplier := NewMessageMetadataSupplier(msgMetadata) + msgMetadataSupplier.SetEncryptionParam(expected) + + assert.EqualValues(t, expected, msgMetadataSupplier.EncryptionParam()) +} diff --git a/pulsar/crypto/testdata/pri_key_rsa.pem b/pulsar/crypto/testdata/pri_key_rsa.pem new file mode 100644 index 0000000000..8b184ffb15 --- /dev/null +++ b/pulsar/crypto/testdata/pri_key_rsa.pem @@ -0,0 +1,39 @@ +-----BEGIN RSA PRIVATE KEY----- +MIIG4wIBAAKCAYEA5YeHRDCRtEn8kVi7xVT2YicRByZZUPjzB9qSlQboHvdIpZhW +hoRg+qPYP2RoH7JlmPX/q/RnNjFZ52pG9Tzl3J67Pz3H5j4FoKFZ7OTf55Wk2f83 +pe9hgDTfAfIuR1ch2MtfOx1yzoVUENFfLwiIhSg/+6wjY98SbZujM1a/iWpM9IFk +pOgwZZrXtSPoUlNidwAZrYS+eGVL9hyyH9RT86gYh1BgsVH4zjQqCP76lxXrsSyn +lsXfasenpTppVfvyR1NhK1JhQf3dsvCgie0iWcK+OICt0z9WesakKnyAU01BFOmX +APuDS2wErTb2Zw654lo7iEFdRGK1Mmx0gViJ7bXBxKVIw24za/EsCeb+jLFXVilC +/P0T7mI/y6FF0wPkTf8aICJphspdnimEJSXeQinjmx+iFWvG/uHimrswXaAQbP1Y +OoJZx+HFy4b/4/hVLu3aiFM2sgfGkT8gsg3di2zxEIn7cnLaIC4HgcVxQLUVMsBX +FXgU2SWVWPb2+fv9AgMBAAECggGAIohVbYjxIvLOsP9soK+7seC2yyCV53zM862K +yCkV2zLRFzjoK6zW+l6UNlVg5QPuDSlVogVPUfPy1sJCkrrqylqHSWh+HsHODvC3 +mtCEb08wUiv3r1toi+Vod9573/fX+8n9NeOkVjXxA+a8L1NVVtNLbQ3k9S2mlB8K +Wrr+bio0EcU14ymbvm2hbntMhLZO5iB5vIVwMqTq3MhMdEV5q/rqVdFd+s+eY0et +21ShVwHRSAKz5Bc5GdOYAV/cYTdaXyR3RJUxy+JT0QW5qJ7FKGa7WIvh2epIdQtf +N79aXQ6ZpQK8aghd6hintV8pJgxD2EcRaT2AAHmuLj1P74YO7WJJnQy/eXhdq2jl +xmTqk9TgFQksvin8QtJyFQzFvxWUKN4z7NAnFlDCM7Yj/k8QnA85kWtyBpBuCFhr +/fapz59Kg5apaAlWKFe9piTezQim8CNgPjdhDjyo7Yyy/rr6XI1ncmC+xKnWpyeH +gXPxDT+7pC1TAU6DC8DRSizh5DGBAoHBAP3JFDSSwwNP5UP9P2EmslhcKR3Ks6rY +XuUotK7yy5sXtkxW2+xGOaKyrCjw4WSXqx0FgGndpejG93bBLM+SwhT/OYZYAqju +f2B9iOFIDshSumpCor19FBn4fnb933RkNS/O9k4iQye6He1L8hleBHMoqDJmFbk2 +vESwNLYrpxuQVfjMbWsLoamwRs2gqwYtd4LvSfwtwxLtTozZhbMufRaTb610VDlZ +iALDTvWRpGjDLaSoz+agaSDalGHVmnfmhQKBwQDniEPMKAiQM6Nk1jsWfFfu0E/0 +0v55F99naAdJuaG0IkJNtD0WCbjtM25gk82hbcP7XP7Sz+ZaNu8Q16xOpOYvtgn5 +Yqd1/5Hh8oQf7lwRPFfpr/PeHjW1f4qCQokPgSut3E8nKKfQKhhRajJH78JBqfPW +Q7A90W4zBC62sTDRqbAzkDq1OKlTu5HyuNGmPqnum8olIgsaOsoBIpsCqfeuFElq +WXCpw8NKOaH8YQP1hd+PhVKrHcNW/EFhhjOqZRkCgcAPcXD9UgDz7qSw4nQ84THx +FoqZ+X+9YbVElJmKG9Qv991r/80aL5vKPr0jMKVGjcQn2/HYf1hdNd5RJ6gmaXPN ++0nw1uIyjXDK2li9/LiJkB8v4CYvCbFzcx+e9gvm9UIXSqzKTGNxw22WxwxQZtw2 +db7mcjfYMXB7bY2HmFhu4PWaUjZGUUrhHIzyblh548JmAVGrOs2oFTC2eXYdVTLf +cNFW6MFHTB9uq5vebaJnjZj1cCBWlGRRT3vACFOCAFECgcAlJdzS3b15/X8Cx8iV +NAAbxfp+Kng/z4+9lJhOwOTr9O80bm26ona0QCM+hZhhhS4Dn4kXI9ousU+sIR55 +Q8XW89sn0ydRLF8opHOEeAb3kPn9+YgkJC6z3zHG8ovxG+V5MLbWbpR2NrrOHT7S +AermBDGmOBgH4xlOQCaKk2VkzlgB/esddmjckWS6T+L7TGSRbxeA27RyUeplQjsi +s0iU+pZI5O7JniowN40A5EPxWbhj251G7TCRPUn0LscNWMkCgcEA6BxNQ907FfQu ++srCpZcmthxKdm8cLrONT/U9cc5tJBftL/zlSvGnhU+rFEt/nLOYoroa9dhZlM8/ +i3BXZofMQ36Q76QIaN41fbzzwpDz5aIFlN/cvASk5Yspdv3Lq1dis0FzAcHFweWU +4jw9m4f9nn81b3QiAjT5aB1Ftqeu9L9r4EZHqAjU+iBN/HpF6W7YPhKKX7VNkX7g +uUZjERpZq5kg7DWWqhcwLc2ztiRrBmT964Q6CnRTzvYmdyaqyQni +-----END RSA PRIVATE KEY----- \ No newline at end of file diff --git a/pulsar/crypto/testdata/pub_key_rsa.pem b/pulsar/crypto/testdata/pub_key_rsa.pem new file mode 100644 index 0000000000..8e49886c5c --- /dev/null +++ b/pulsar/crypto/testdata/pub_key_rsa.pem @@ -0,0 +1,11 @@ +-----BEGIN PUBLIC KEY----- +MIIBojANBgkqhkiG9w0BAQEFAAOCAY8AMIIBigKCAYEA5YeHRDCRtEn8kVi7xVT2 +YicRByZZUPjzB9qSlQboHvdIpZhWhoRg+qPYP2RoH7JlmPX/q/RnNjFZ52pG9Tzl +3J67Pz3H5j4FoKFZ7OTf55Wk2f83pe9hgDTfAfIuR1ch2MtfOx1yzoVUENFfLwiI +hSg/+6wjY98SbZujM1a/iWpM9IFkpOgwZZrXtSPoUlNidwAZrYS+eGVL9hyyH9RT +86gYh1BgsVH4zjQqCP76lxXrsSynlsXfasenpTppVfvyR1NhK1JhQf3dsvCgie0i +WcK+OICt0z9WesakKnyAU01BFOmXAPuDS2wErTb2Zw654lo7iEFdRGK1Mmx0gViJ +7bXBxKVIw24za/EsCeb+jLFXVilC/P0T7mI/y6FF0wPkTf8aICJphspdnimEJSXe +Qinjmx+iFWvG/uHimrswXaAQbP1YOoJZx+HFy4b/4/hVLu3aiFM2sgfGkT8gsg3d +i2zxEIn7cnLaIC4HgcVxQLUVMsBXFXgU2SWVWPb2+fv9AgMBAAE= +-----END PUBLIC KEY----- \ No newline at end of file diff --git a/pulsar/crypto/testdata/truncated_pub_key_rsa.pem b/pulsar/crypto/testdata/truncated_pub_key_rsa.pem new file mode 100644 index 0000000000..51a0381b9d --- /dev/null +++ b/pulsar/crypto/testdata/truncated_pub_key_rsa.pem @@ -0,0 +1,10 @@ +-----BEGIN PUBLIC KEY----- +MIIBojANBgkqhkiG9w0BAQEFAAOCAY8AMIIBigKCAYEA5YeHRDCRtEn8kVi7xVT2 +YicRByZZUPjzB9qSlQboHvdIpZhWhoRg+qPYP2RoH7JlmPX/q/RnNjFZ52pG9Tzl +3J67Pz3H5j4FoKFZ7OTf55Wk2f83pe9hgDTfAfIuR1ch2MtfOx1yzoVUENFfLwiI +hSg/+6wjY98SbZujM1a/iWpM9IFkpOgwZZrXtSPoUlNidwAZrYS+eGVL9hyyH9RT +86gYh1BgsVH4zjQqCP76lxXrsSynlsXfasenpTppVfvyR1NhK1JhQf3dsvCgie0i +WcK+OICt0z9WesakKnyAU01BFOmXAPuDS2wErTb2Zw654lo7iEFdRGK1Mmx0gViJ +7bXBxKVIw24za/EsCeb+jLFXVilC/P0T7mI/y6FF0wPkTf8aICJphspdnimEJSXe +Qinjmx+iFWvG/uHimrswXaAQbP1YOoJZx+HFy4b/4/hVLu3aiFM2sgfGkT8gsg3d +-----END PUBLIC KEY----- \ No newline at end of file diff --git a/pulsar/crypto/testdata/wrong_encode_pub_key_rsa.pem b/pulsar/crypto/testdata/wrong_encode_pub_key_rsa.pem new file mode 100644 index 0000000000..3d40cc288a --- /dev/null +++ b/pulsar/crypto/testdata/wrong_encode_pub_key_rsa.pem @@ -0,0 +1,9 @@ +-----BEGIN PUBLIC KEY----- +MIIBojANBgkqhkiG9w0BAQEFAAOCAY8AMIIBigKCAYEA5YeHRDCRtEn8kVi7xVT2 +YicRByZZUPjzB9qSlQboHvdIpZhWhoRg+qPYP2RoH7JlmPX/q/RnNjFZ52pG9Tzl +3J67Pz3H5j4FoKFZ7OTf55Wk2f83pe9hgDTfAfIuR1ch2MtfOx1yzoVUENFfLwiI +hSg/+6wjY98SbZujM1a/iWpM9IFkpOgwZZrXtSPoUlNidwAZrYS+eGVL9hyyH9RT +86gYh1BgsVH4zjQqCP76lxXrsSynlsXfasenpTppVfvyR1NhK1JhQf3dsvCgie0i +WcK+OICt0z9WesakKnyAU01BFOmXAPuDS2wErTb2Zw654lo7iEFdRGK1Mmx0gViJ +7bXBxKVIw24za/EsCeb+jLFXVilC/P0T7mI/y6FF0wPkTf8aICJphspdnimEJSXe +Qinjmx+iFWvG/uHimrswXaAQbP1YOoJZx+HFy4b/4/hVLu3aiFM2sgfGkT8gsg3d \ No newline at end of file diff --git a/pulsar/crypto/testdata/wrong_encoded_pri_key_rsa.pem b/pulsar/crypto/testdata/wrong_encoded_pri_key_rsa.pem new file mode 100644 index 0000000000..2566ae3590 --- /dev/null +++ b/pulsar/crypto/testdata/wrong_encoded_pri_key_rsa.pem @@ -0,0 +1,38 @@ +-----BEGIN RSA PRIVATE KEY----- +MIIG4wIBAAKCAYEA5YeHRDCRtEn8kVi7xVT2YicRByZZUPjzB9qSlQboHvdIpZhW +hoRg+qPYP2RoH7JlmPX/q/RnNjFZ52pG9Tzl3J67Pz3H5j4FoKFZ7OTf55Wk2f83 +pe9hgDTfAfIuR1ch2MtfOx1yzoVUENFfLwiIhSg/+6wjY98SbZujM1a/iWpM9IFk +pOgwZZrXtSPoUlNidwAZrYS+eGVL9hyyH9RT86gYh1BgsVH4zjQqCP76lxXrsSyn +lsXfasenpTppVfvyR1NhK1JhQf3dsvCgie0iWcK+OICt0z9WesakKnyAU01BFOmX +APuDS2wErTb2Zw654lo7iEFdRGK1Mmx0gViJ7bXBxKVIw24za/EsCeb+jLFXVilC +/P0T7mI/y6FF0wPkTf8aICJphspdnimEJSXeQinjmx+iFWvG/uHimrswXaAQbP1Y +OoJZx+HFy4b/4/hVLu3aiFM2sgfGkT8gsg3di2zxEIn7cnLaIC4HgcVxQLUVMsBX +FXgU2SWVWPb2+fv9AgMBAAECggGAIohVbYjxIvLOsP9soK+7seC2yyCV53zM862K +yCkV2zLRFzjoK6zW+l6UNlVg5QPuDSlVogVPUfPy1sJCkrrqylqHSWh+HsHODvC3 +mtCEb08wUiv3r1toi+Vod9573/fX+8n9NeOkVjXxA+a8L1NVVtNLbQ3k9S2mlB8K +Wrr+bio0EcU14ymbvm2hbntMhLZO5iB5vIVwMqTq3MhMdEV5q/rqVdFd+s+eY0et +21ShVwHRSAKz5Bc5GdOYAV/cYTdaXyR3RJUxy+JT0QW5qJ7FKGa7WIvh2epIdQtf +N79aXQ6ZpQK8aghd6hintV8pJgxD2EcRaT2AAHmuLj1P74YO7WJJnQy/eXhdq2jl +xmTqk9TgFQksvin8QtJyFQzFvxWUKN4z7NAnFlDCM7Yj/k8QnA85kWtyBpBuCFhr +/fapz59Kg5apaAlWKFe9piTezQim8CNgPjdhDjyo7Yyy/rr6XI1ncmC+xKnWpyeH +gXPxDT+7pC1TAU6DC8DRSizh5DGBAoHBAP3JFDSSwwNP5UP9P2EmslhcKR3Ks6rY +XuUotK7yy5sXtkxW2+xGOaKyrCjw4WSXqx0FgGndpejG93bBLM+SwhT/OYZYAqju +f2B9iOFIDshSumpCor19FBn4fnb933RkNS/O9k4iQye6He1L8hleBHMoqDJmFbk2 +vESwNLYrpxuQVfjMbWsLoamwRs2gqwYtd4LvSfwtwxLtTozZhbMufRaTb610VDlZ +iALDTvWRpGjDLaSoz+agaSDalGHVmnfmhQKBwQDniEPMKAiQM6Nk1jsWfFfu0E/0 +0v55F99naAdJuaG0IkJNtD0WCbjtM25gk82hbcP7XP7Sz+ZaNu8Q16xOpOYvtgn5 +Yqd1/5Hh8oQf7lwRPFfpr/PeHjW1f4qCQokPgSut3E8nKKfQKhhRajJH78JBqfPW +Q7A90W4zBC62sTDRqbAzkDq1OKlTu5HyuNGmPqnum8olIgsaOsoBIpsCqfeuFElq +WXCpw8NKOaH8YQP1hd+PhVKrHcNW/EFhhjOqZRkCgcAPcXD9UgDz7qSw4nQ84THx +FoqZ+X+9YbVElJmKG9Qv991r/80aL5vKPr0jMKVGjcQn2/HYf1hdNd5RJ6gmaXPN ++0nw1uIyjXDK2li9/LiJkB8v4CYvCbFzcx+e9gvm9UIXSqzKTGNxw22WxwxQZtw2 +db7mcjfYMXB7bY2HmFhu4PWaUjZGUUrhHIzyblh548JmAVGrOs2oFTC2eXYdVTLf +cNFW6MFHTB9uq5vebaJnjZj1cCBWlGRRT3vACFOCAFECgcAlJdzS3b15/X8Cx8iV +NAAbxfp+Kng/z4+9lJhOwOTr9O80bm26ona0QCM+hZhhhS4Dn4kXI9ousU+sIR55 +Q8XW89sn0ydRLF8opHOEeAb3kPn9+YgkJC6z3zHG8ovxG+V5MLbWbpR2NrrOHT7S +AermBDGmOBgH4xlOQCaKk2VkzlgB/esddmjckWS6T+L7TGSRbxeA27RyUeplQjsi +s0iU+pZI5O7JniowN40A5EPxWbhj251G7TCRPUn0LscNWMkCgcEA6BxNQ907FfQu ++srCpZcmthxKdm8cLrONT/U9cc5tJBftL/zlSvGnhU+rFEt/nLOYoroa9dhZlM8/ +i3BXZofMQ36Q76QIaN41fbzzwpDz5aIFlN/cvASk5Yspdv3Lq1dis0FzAcHFweWU +4jw9m4f9nn81b3QiAjT5aB1Ftqeu9L9r4EZHqAjU+iBN/HpF6W7YPhKKX7VNkX7g +uUZjERpZq5kg7DWWqhcwLc2ztiRrBmT964Q6CnRTzvYmdyaqyQni \ No newline at end of file From 0de47927ca02e5567074f557063ad128da4e3125 Mon Sep 17 00:00:00 2001 From: Marais <84397028+maraiskruger1980@users.noreply.github.com> Date: Mon, 12 Jul 2021 10:25:40 +0300 Subject: [PATCH 18/25] Add open tracing to pulsar go clinet (#518) Here is the PR to add open tracing to the pulsar go client directly. This is replicating what the java tracer does: https://github.com/streamnative/pulsar-tracing The usage is stipulated in the readme.md file Co-authored-by: Marais Kruger --- go.mod | 3 +- go.sum | 2 + .../pulsartracing/consumer_interceptor.go | 58 +++++++++ .../consumer_interceptor_test.go | 89 +++++++++++++ .../pulsartracing/message_carrier_adaptors.go | 84 ++++++++++++ .../pulsartracing/message_carrier_util.go | 89 +++++++++++++ .../message_carrier_util_test.go | 121 ++++++++++++++++++ .../pulsartracing/producer_interceptor.go | 57 +++++++++ .../producer_interceptor_test.go | 69 ++++++++++ pulsar/internal/pulsartracing/readme.md | 40 ++++++ .../internal/pulsartracing/span-enrichment.go | 50 ++++++++ 11 files changed, 660 insertions(+), 2 deletions(-) create mode 100644 pulsar/internal/pulsartracing/consumer_interceptor.go create mode 100644 pulsar/internal/pulsartracing/consumer_interceptor_test.go create mode 100644 pulsar/internal/pulsartracing/message_carrier_adaptors.go create mode 100644 pulsar/internal/pulsartracing/message_carrier_util.go create mode 100644 pulsar/internal/pulsartracing/message_carrier_util_test.go create mode 100644 pulsar/internal/pulsartracing/producer_interceptor.go create mode 100644 pulsar/internal/pulsartracing/producer_interceptor_test.go create mode 100644 pulsar/internal/pulsartracing/readme.md create mode 100644 pulsar/internal/pulsartracing/span-enrichment.go diff --git a/go.mod b/go.mod index 6d53cc6028..67a7bb725b 100644 --- a/go.mod +++ b/go.mod @@ -16,8 +16,7 @@ require ( github.com/klauspost/compress v1.10.8 github.com/kr/pretty v0.2.0 // indirect github.com/linkedin/goavro/v2 v2.9.8 - github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect - github.com/modern-go/reflect2 v1.0.1 // indirect + github.com/opentracing/opentracing-go v1.2.0 github.com/pierrec/lz4 v2.0.5+incompatible github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.7.1 diff --git a/go.sum b/go.sum index b2818eec4e..2ccd39eb36 100644 --- a/go.sum +++ b/go.sum @@ -125,6 +125,8 @@ github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9k github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= +github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= +github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/pulsar/internal/pulsartracing/consumer_interceptor.go b/pulsar/internal/pulsartracing/consumer_interceptor.go new file mode 100644 index 0000000000..3969d45d8b --- /dev/null +++ b/pulsar/internal/pulsartracing/consumer_interceptor.go @@ -0,0 +1,58 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package pulsartracing + +import ( + "context" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/opentracing/opentracing-go" +) + +const fromPrefix = "From__" + +type ConsumerInterceptor struct { +} + +func (t *ConsumerInterceptor) BeforeConsume(message pulsar.ConsumerMessage) { + buildAndInjectChildSpan(message).Finish() +} + +func (t *ConsumerInterceptor) OnAcknowledge(consumer pulsar.Consumer, msgID pulsar.MessageID) {} + +func (t *ConsumerInterceptor) OnNegativeAcksSend(consumer pulsar.Consumer, msgIDs []pulsar.MessageID) { +} + +func buildAndInjectChildSpan(message pulsar.ConsumerMessage) opentracing.Span { + tracer := opentracing.GlobalTracer() + parentContext := ExtractSpanContextFromConsumerMessage(message) + + var span opentracing.Span + + var startSpanOptions []opentracing.StartSpanOption + if parentContext != nil { + startSpanOptions = []opentracing.StartSpanOption{opentracing.FollowsFrom(parentContext)} + } + + span = tracer.StartSpan(fromPrefix+message.Topic()+"__"+message.Subscription(), startSpanOptions...) + + enrichConsumerSpan(&message, span) + InjectConsumerMessageSpanContext(opentracing.ContextWithSpan(context.Background(), span), message) + + return span +} diff --git a/pulsar/internal/pulsartracing/consumer_interceptor_test.go b/pulsar/internal/pulsartracing/consumer_interceptor_test.go new file mode 100644 index 0000000000..b15a926bee --- /dev/null +++ b/pulsar/internal/pulsartracing/consumer_interceptor_test.go @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package pulsartracing + +import ( + "context" + "testing" + "time" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/opentracing/opentracing-go" + "github.com/opentracing/opentracing-go/mocktracer" + "github.com/stretchr/testify/assert" +) + +func TestConsumerBuildAndInjectChildSpan(t *testing.T) { + tracer := mocktracer.New() + + opentracing.SetGlobalTracer(tracer) + + message := pulsar.ConsumerMessage{ + Consumer: &mockConsumer{}, + Message: &mockConsumerMessage{ + properties: map[string]string{}, + }, + } + + span := buildAndInjectChildSpan(message) + assert.NotNil(t, span) + assert.True(t, len(message.Properties()) > 0) +} + +type mockConsumer struct { +} + +func (c *mockConsumer) Subscription() string { + return "" +} + +func (c *mockConsumer) Unsubscribe() error { + return nil +} + +func (c *mockConsumer) Receive(ctx context.Context) (message pulsar.Message, err error) { + return nil, nil +} + +func (c *mockConsumer) Chan() <-chan pulsar.ConsumerMessage { + return nil +} + +func (c *mockConsumer) Ack(msg pulsar.Message) {} + +func (c *mockConsumer) AckID(msgID pulsar.MessageID) {} + +func (c *mockConsumer) ReconsumeLater(msg pulsar.Message, delay time.Duration) {} + +func (c *mockConsumer) Nack(msg pulsar.Message) {} + +func (c *mockConsumer) NackID(msgID pulsar.MessageID) {} + +func (c *mockConsumer) Close() {} + +func (c *mockConsumer) Seek(msgID pulsar.MessageID) error { + return nil +} + +func (c *mockConsumer) SeekByTime(time time.Time) error { + return nil +} + +func (c *mockConsumer) Name() string { + return "" +} diff --git a/pulsar/internal/pulsartracing/message_carrier_adaptors.go b/pulsar/internal/pulsartracing/message_carrier_adaptors.go new file mode 100644 index 0000000000..3b6394baa7 --- /dev/null +++ b/pulsar/internal/pulsartracing/message_carrier_adaptors.go @@ -0,0 +1,84 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package pulsartracing + +import ( + "errors" + + "github.com/apache/pulsar-client-go/pulsar" +) + +// ProducerMessageExtractAdapter Implements TextMap Interface +type ProducerMessageExtractAdapter struct { + message *pulsar.ProducerMessage +} + +func (a *ProducerMessageExtractAdapter) ForeachKey(handler func(key, val string) error) error { + for k, v := range (*a.message).Properties { + if err := handler(k, v); err != nil { + return err + } + } + + return nil +} + +func (a *ProducerMessageExtractAdapter) Set(key, val string) {} + +// ProducerMessageInjectAdapter Implements TextMap Interface +type ProducerMessageInjectAdapter struct { + message *pulsar.ProducerMessage +} + +func (a *ProducerMessageInjectAdapter) ForeachKey(handler func(key, val string) error) error { + return errors.New("iterator should never be used with Tracer.inject()") +} + +func (a *ProducerMessageInjectAdapter) Set(key, val string) { + a.message.Properties[key] = val +} + +// ConsumerMessageExtractAdapter Implements TextMap Interface +type ConsumerMessageExtractAdapter struct { + message pulsar.ConsumerMessage +} + +func (a *ConsumerMessageExtractAdapter) ForeachKey(handler func(key, val string) error) error { + for k, v := range a.message.Properties() { + if err := handler(k, v); err != nil { + return err + } + } + + return nil +} + +func (a *ConsumerMessageExtractAdapter) Set(key, val string) {} + +// ConsumerMessageInjectAdapter Implements TextMap Interface +type ConsumerMessageInjectAdapter struct { + message pulsar.ConsumerMessage +} + +func (a *ConsumerMessageInjectAdapter) ForeachKey(handler func(key, val string) error) error { + return errors.New("iterator should never be used with tracer.inject()") +} + +func (a *ConsumerMessageInjectAdapter) Set(key, val string) { + a.message.Properties()[key] = val +} diff --git a/pulsar/internal/pulsartracing/message_carrier_util.go b/pulsar/internal/pulsartracing/message_carrier_util.go new file mode 100644 index 0000000000..d1fd0ddb96 --- /dev/null +++ b/pulsar/internal/pulsartracing/message_carrier_util.go @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package pulsartracing + +import ( + "context" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/opentracing/opentracing-go" + log "github.com/sirupsen/logrus" +) + +func InjectProducerMessageSpanContext(ctx context.Context, message *pulsar.ProducerMessage) { + injectAdapter := &ProducerMessageInjectAdapter{message} + + span := opentracing.SpanFromContext(ctx) + + err := opentracing.GlobalTracer().Inject(span.Context(), opentracing.TextMap, injectAdapter) + + if err != nil { + log.Error("could not inject span context into pulsar message", err) + } +} + +func ExtractSpanContextFromProducerMessage(message *pulsar.ProducerMessage) opentracing.SpanContext { + extractAdapter := &ProducerMessageExtractAdapter{message} + + spanContext, err := opentracing.GlobalTracer().Extract(opentracing.TextMap, extractAdapter) + + if err != nil { + log.Error("could not extract span context from pulsar message", err) + } + + return spanContext +} + +func ExtractSpanContextFromConsumerMessage(message pulsar.ConsumerMessage) opentracing.SpanContext { + extractAdapter := &ConsumerMessageExtractAdapter{message} + + spanContext, err := opentracing.GlobalTracer().Extract(opentracing.TextMap, extractAdapter) + + if err != nil { + log.Error("could not extract span context from pulsar message", err) + } + + return spanContext +} + +func InjectConsumerMessageSpanContext(ctx context.Context, message pulsar.ConsumerMessage) { + injectAdapter := &ConsumerMessageInjectAdapter{message} + span := opentracing.SpanFromContext(ctx) + + if span == nil { + log.Warn("no span could be extracted from context, nothing will be injected into the message properties") + return + } + + err := opentracing.GlobalTracer().Inject(span.Context(), opentracing.TextMap, injectAdapter) + + if err != nil { + log.Error("could not inject span context into pulsar message", err) + } +} + +func CreateSpanFromMessage(cm *pulsar.ConsumerMessage, tracer opentracing.Tracer, label string) opentracing.Span { + parentSpan := ExtractSpanContextFromConsumerMessage(*cm) + var span opentracing.Span + if parentSpan != nil { + span = tracer.StartSpan(label, opentracing.ChildOf(parentSpan)) + } else { + span = tracer.StartSpan(label) + } + return span +} diff --git a/pulsar/internal/pulsartracing/message_carrier_util_test.go b/pulsar/internal/pulsartracing/message_carrier_util_test.go new file mode 100644 index 0000000000..9fe608dc90 --- /dev/null +++ b/pulsar/internal/pulsartracing/message_carrier_util_test.go @@ -0,0 +1,121 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package pulsartracing + +import ( + "context" + "testing" + "time" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/opentracing/opentracing-go" + "github.com/opentracing/opentracing-go/mocktracer" + "github.com/stretchr/testify/assert" +) + +func TestProducerMessageInjectAndExtract(t *testing.T) { + message := &pulsar.ProducerMessage{ + Properties: map[string]string{}, + } + + tracer := mocktracer.New() + + opentracing.SetGlobalTracer(tracer) + + span := tracer.StartSpan("test") + + InjectProducerMessageSpanContext(opentracing.ContextWithSpan(context.Background(), span), message) + assert.True(t, len(message.Properties) > 0) + extractedSpanContext := ExtractSpanContextFromProducerMessage(message) + assert.Equal(t, span.Context(), extractedSpanContext) +} + +func TestConsumerMessageInjectAndExtract(t *testing.T) { + message := pulsar.ConsumerMessage{ + Message: &mockConsumerMessage{ + properties: map[string]string{}, + }, + } + + tracer := mocktracer.New() + + opentracing.SetGlobalTracer(tracer) + + span := tracer.StartSpan("test") + + InjectConsumerMessageSpanContext(opentracing.ContextWithSpan(context.Background(), span), message) + assert.True(t, len(message.Properties()) > 0) + extractedSpanContext := ExtractSpanContextFromConsumerMessage(message) + assert.Equal(t, span.Context(), extractedSpanContext) +} + +type mockConsumerMessage struct { + properties map[string]string +} + +func (msg *mockConsumerMessage) Topic() string { + return "" +} + +func (msg *mockConsumerMessage) Properties() map[string]string { + return msg.properties +} + +func (msg *mockConsumerMessage) Payload() []byte { + return nil +} + +func (msg *mockConsumerMessage) ID() pulsar.MessageID { + return nil +} + +func (msg *mockConsumerMessage) PublishTime() time.Time { + return time.Time{} +} + +func (msg *mockConsumerMessage) EventTime() time.Time { + return time.Time{} +} + +func (msg *mockConsumerMessage) Key() string { + return "" +} + +func (msg *mockConsumerMessage) OrderingKey() string { + return "" +} + +func (msg *mockConsumerMessage) RedeliveryCount() uint32 { + return 0 +} + +func (msg *mockConsumerMessage) IsReplicated() bool { + return false +} + +func (msg *mockConsumerMessage) GetReplicatedFrom() string { + return "" +} + +func (msg *mockConsumerMessage) GetSchemaValue(v interface{}) error { + return nil +} + +func (msg *mockConsumerMessage) ProducerName() string { + return "" +} diff --git a/pulsar/internal/pulsartracing/producer_interceptor.go b/pulsar/internal/pulsartracing/producer_interceptor.go new file mode 100644 index 0000000000..b400e57785 --- /dev/null +++ b/pulsar/internal/pulsartracing/producer_interceptor.go @@ -0,0 +1,57 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package pulsartracing + +import ( + "context" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/opentracing/opentracing-go" +) + +const toPrefix = "To__" + +type ProducerInterceptor struct { +} + +func (t *ProducerInterceptor) BeforeSend(producer pulsar.Producer, message *pulsar.ProducerMessage) { + buildAndInjectSpan(message, producer).Finish() +} + +func (t *ProducerInterceptor) OnSendAcknowledgement(producer pulsar.Producer, message *pulsar.ProducerMessage, msgID pulsar.MessageID) { +} + +func buildAndInjectSpan(message *pulsar.ProducerMessage, producer pulsar.Producer) opentracing.Span { + tracer := opentracing.GlobalTracer() + spanContext := ExtractSpanContextFromProducerMessage(message) + + var span opentracing.Span + + var startSpanOptions []opentracing.StartSpanOption + if spanContext != nil { + startSpanOptions = []opentracing.StartSpanOption{opentracing.FollowsFrom(spanContext)} + } + + span = tracer.StartSpan(toPrefix+producer.Topic(), startSpanOptions...) + + enrichProducerSpan(message, producer, span) + + InjectProducerMessageSpanContext(opentracing.ContextWithSpan(context.Background(), span), message) + + return span +} diff --git a/pulsar/internal/pulsartracing/producer_interceptor_test.go b/pulsar/internal/pulsartracing/producer_interceptor_test.go new file mode 100644 index 0000000000..b146e4e93b --- /dev/null +++ b/pulsar/internal/pulsartracing/producer_interceptor_test.go @@ -0,0 +1,69 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package pulsartracing + +import ( + "context" + "testing" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/opentracing/opentracing-go" + "github.com/opentracing/opentracing-go/mocktracer" + "github.com/stretchr/testify/assert" +) + +func TestProducerBuildAndInjectSpan(t *testing.T) { + tracer := mocktracer.New() + opentracing.SetGlobalTracer(tracer) + + message := &pulsar.ProducerMessage{ + Properties: map[string]string{}, + } + + span := buildAndInjectSpan(message, &mockProducer{}) + assert.NotNil(t, span) + assert.True(t, len(message.Properties) > 0) +} + +type mockProducer struct { +} + +func (p *mockProducer) Topic() string { + return "" +} + +func (p *mockProducer) Name() string { + return "" +} + +func (p *mockProducer) Send(context.Context, *pulsar.ProducerMessage) (pulsar.MessageID, error) { + return nil, nil +} + +func (p *mockProducer) SendAsync(context.Context, *pulsar.ProducerMessage, func(pulsar.MessageID, *pulsar.ProducerMessage, error)) { +} + +func (p *mockProducer) LastSequenceID() int64 { + return 0 +} + +func (p *mockProducer) Flush() error { + return nil +} + +func (p *mockProducer) Close() {} diff --git a/pulsar/internal/pulsartracing/readme.md b/pulsar/internal/pulsartracing/readme.md new file mode 100644 index 0000000000..319587e436 --- /dev/null +++ b/pulsar/internal/pulsartracing/readme.md @@ -0,0 +1,40 @@ +### Usage + +#### Interceptors based solution + +```go +// create new tracer +// register tracer with GlobalTracer +opentracing.SetGlobalTracer(tracer) +``` + +**Producer** + +```go +tracingInterceptor := &pulsartracing.ProducerInterceptor{} + +options := pulsar.ProducerOptions{ +Topic: topicName, +Interceptors: pulsar.ProducerInterceptors{tracingInterceptor}, +} +``` + +**Consumer** +```go +tracingInterceptor := &pulsartracing.ConsumerInterceptor{} + +options := pulsar.ConsumerOptions{ +Topics: topicName, +SubscriptionName: subscriptionName, +Type: pulsar.Shared, +Interceptors: pulsar.ConsumerInterceptors{tracingInterceptor}, +} + + +// to create span with message as parent span +span := pulsartracing.CreateSpanFromMessage(message, tracer, "child_span") +``` + +## License + +[Apache 2.0 License](./LICENSE). \ No newline at end of file diff --git a/pulsar/internal/pulsartracing/span-enrichment.go b/pulsar/internal/pulsartracing/span-enrichment.go new file mode 100644 index 0000000000..e75c398813 --- /dev/null +++ b/pulsar/internal/pulsartracing/span-enrichment.go @@ -0,0 +1,50 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package pulsartracing + +import ( + "github.com/apache/pulsar-client-go/pulsar" + "github.com/opentracing/opentracing-go" +) + +func enrichConsumerSpan(message *pulsar.ConsumerMessage, span opentracing.Span) { + spanCommonTags(span) + + for k, v := range message.Properties() { + span.SetTag(k, v) + } + span.SetTag("message_bus.destination", message.Topic()) + span.SetTag("messageId", message.ID()) + span.SetTag("subscription", message.Subscription()) +} + +func enrichProducerSpan(message *pulsar.ProducerMessage, producer pulsar.Producer, span opentracing.Span) { + spanCommonTags(span) + + for k, v := range message.Properties { + span.SetTag(k, v) + } + span.SetTag("span.kind", "producer") + span.SetTag("message_bus.destination", producer.Topic()) + span.SetTag("sequenceId", producer.LastSequenceID()) +} + +func spanCommonTags(span opentracing.Span) { + span.SetTag("component", "pulsar-client-go") + span.SetTag("peer.service", "pulsar-broker") +} From 9d864c224e0943cf1cba95ec3f967211edc4f0cb Mon Sep 17 00:00:00 2001 From: cckellogg Date: Mon, 12 Jul 2021 03:30:17 -0400 Subject: [PATCH 19/25] Fix possible race condition in connection pool (#561) * Fix possible race condition in connection pool - Use mutex instead of sync map in connection pool - Create tickers in run function for the connection instead of when the connection is created * Remove commented out code --- pulsar/internal/connection.go | 41 +++++++++------- pulsar/internal/connection_pool.go | 77 +++++++++++++++--------------- 2 files changed, 62 insertions(+), 56 deletions(-) diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 2549e21991..a632816a16 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -128,6 +128,7 @@ type incomingCmd struct { type connection struct { sync.Mutex cond *sync.Cond + started int32 state ua.Int32 connectionTimeout time.Duration closeOnce sync.Once @@ -142,8 +143,6 @@ type connection struct { lastDataReceivedLock sync.Mutex lastDataReceivedTime time.Time - pingTicker *time.Ticker - pingCheckTicker *time.Ticker log log.Logger @@ -191,8 +190,6 @@ func newConnection(opts connectionOptions) *connection { log: opts.logger.SubLogger(log.Fields{"remote_addr": opts.physicalAddr}), pendingReqs: make(map[uint64]*request), lastDataReceivedTime: time.Now(), - pingTicker: time.NewTicker(keepAliveInterval), - pingCheckTicker: time.NewTicker(keepAliveInterval), tlsOptions: opts.tls, auth: opts.auth, @@ -217,6 +214,11 @@ func newConnection(opts connectionOptions) *connection { } func (c *connection) start() { + if !atomic.CompareAndSwapInt32(&c.started, 0, 1) { + c.log.Warnf("connection has already started") + return + } + // Each connection gets its own goroutine that will go func() { if c.connect() { @@ -354,16 +356,17 @@ func (c *connection) failLeftRequestsWhenClose() { } func (c *connection) run() { - // All reads come from the reader goroutine - go c.reader.readFromConnection() - go c.runPingCheck() - - c.log.Debugf("Connection run starting with request capacity=%d queued=%d", - cap(c.incomingRequestsCh), len(c.incomingRequestsCh)) + pingSendTicker := time.NewTicker(keepAliveInterval) + pingCheckTicker := time.NewTicker(keepAliveInterval) defer func() { + // stop tickers + pingSendTicker.Stop() + pingCheckTicker.Stop() + // all the accesses to the pendingReqs should be happened in this run loop thread, - // including the final cleanup, to avoid the issue https://github.com/apache/pulsar-client-go/issues/239 + // including the final cleanup, to avoid the issue + // https://github.com/apache/pulsar-client-go/issues/239 c.pendingLock.Lock() for id, req := range c.pendingReqs { req.callback(nil, errConnectionClosed) @@ -373,6 +376,13 @@ func (c *connection) run() { c.Close() }() + // All reads come from the reader goroutine + go c.reader.readFromConnection() + go c.runPingCheck(pingCheckTicker) + + c.log.Debugf("Connection run starting with request capacity=%d queued=%d", + cap(c.incomingRequestsCh), len(c.incomingRequestsCh)) + go func() { for { select { @@ -402,18 +412,18 @@ func (c *connection) run() { } c.internalWriteData(data) - case <-c.pingTicker.C: + case <-pingSendTicker.C: c.sendPing() } } } -func (c *connection) runPingCheck() { +func (c *connection) runPingCheck(pingCheckTicker *time.Ticker) { for { select { case <-c.closeCh: return - case <-c.pingCheckTicker.C: + case <-pingCheckTicker.C: if c.lastDataReceived().Add(2 * keepAliveInterval).Before(time.Now()) { // We have not received a response to the previous Ping request, the // connection to broker is stale @@ -803,9 +813,6 @@ func (c *connection) Close() { close(c.closeCh) - c.pingTicker.Stop() - c.pingCheckTicker.Stop() - listeners := make(map[uint64]ConnectionListener) c.listenersLock.Lock() for id, listener := range c.listeners { diff --git a/pulsar/internal/connection_pool.go b/pulsar/internal/connection_pool.go index 29e126723d..2728a73419 100644 --- a/pulsar/internal/connection_pool.go +++ b/pulsar/internal/connection_pool.go @@ -38,7 +38,8 @@ type ConnectionPool interface { } type connectionPool struct { - pool sync.Map + sync.Mutex + connections map[string]*connection connectionTimeout time.Duration tlsOptions *TLSOptions auth auth.Provider @@ -58,6 +59,7 @@ func NewConnectionPool( logger log.Logger, metrics *Metrics) ConnectionPool { return &connectionPool{ + connections: make(map[string]*connection), tlsOptions: tlsOptions, auth: auth, connectionTimeout: connectionTimeout, @@ -69,54 +71,51 @@ func NewConnectionPool( func (p *connectionPool) GetConnection(logicalAddr *url.URL, physicalAddr *url.URL) (Connection, error) { key := p.getMapKey(logicalAddr) - cachedCnx, found := p.pool.Load(key) - if found { - cnx := cachedCnx.(*connection) - p.log.Debug("Found connection in cache:", cnx.logicalAddr, cnx.physicalAddr) - if err := cnx.waitUntilReady(); err == nil { - // Connection is ready to be used - return cnx, nil + p.Lock() + conn, ok := p.connections[key] + if ok { + p.log.Debugf("Found connection in pool key=%s logical_addr=%+v physical_addr=%+v", + key, conn.logicalAddr, conn.physicalAddr) + + // remove stale/failed connection + if conn.closed() { + delete(p.connections, key) + p.log.Debugf("Removed connection from pool key=%s logical_addr=%+v physical_addr=%+v", + key, conn.logicalAddr, conn.physicalAddr) + conn = nil // set to nil so we create a new one } - // The cached connection is failed - p.pool.Delete(key) - p.log.Debug("Removed failed connection from pool:", cnx.logicalAddr, cnx.physicalAddr) } - // Try to create a new connection - newConnection := newConnection(connectionOptions{ - logicalAddr: logicalAddr, - physicalAddr: physicalAddr, - tls: p.tlsOptions, - connectionTimeout: p.connectionTimeout, - auth: p.auth, - logger: p.log, - metrics: p.metrics, - }) - newCnx, wasCached := p.pool.LoadOrStore(key, newConnection) - cnx := newCnx.(*connection) - - if !wasCached { - cnx.start() + if conn == nil { + conn = newConnection(connectionOptions{ + logicalAddr: logicalAddr, + physicalAddr: physicalAddr, + tls: p.tlsOptions, + connectionTimeout: p.connectionTimeout, + auth: p.auth, + logger: p.log, + metrics: p.metrics, + }) + p.connections[key] = conn + p.Unlock() + conn.start() } else { - newConnection.Close() + // we already have a connection + p.Unlock() } - if err := cnx.waitUntilReady(); err != nil { - if !wasCached { - p.pool.Delete(key) - p.log.Debug("Removed failed connection from pool:", cnx.logicalAddr, cnx.physicalAddr) - } - return nil, err - } - return cnx, nil + err := conn.waitUntilReady() + return conn, err } func (p *connectionPool) Close() { - p.pool.Range(func(key, value interface{}) bool { - value.(Connection).Close() - return true - }) + p.Lock() + for k, c := range p.connections { + delete(p.connections, k) + c.Close() + } + p.Unlock() } func (p *connectionPool) getMapKey(addr *url.URL) string { From d39558319a259709643f3ee8727eed325901dc9d Mon Sep 17 00:00:00 2001 From: xiaolong ran Date: Mon, 12 Jul 2021 15:49:18 +0800 Subject: [PATCH 20/25] Fix default connection timeout (#563) Signed-off-by: xiaolongran ### Motivation In Java SDK, the default connection timeout is **10** sec ``` private int connectionTimeoutMs = 10000; ``` The problem here is that when the load of the broker is high, there is no way to complete the processing and creation of the connection within 5s, which will cause the client to constantly create new connections and try to establish a connection with the broker, and the broker Continuously disconnecting from the client, forming a vicious circle ### Modifications Replace 5 sec of connection timeout with `10` sec --- pulsar/client_impl.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index 52ef5b1ca7..7d2fcfd972 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -30,7 +30,7 @@ import ( ) const ( - defaultConnectionTimeout = 5 * time.Second + defaultConnectionTimeout = 10 * time.Second defaultOperationTimeout = 30 * time.Second ) From 7e16f6b3da1173a5b41600f9d022a02eacf42368 Mon Sep 17 00:00:00 2001 From: xiaolong ran Date: Mon, 12 Jul 2021 15:49:38 +0800 Subject: [PATCH 21/25] Update proto file (#562) Signed-off-by: xiaolongran ### Motivation The current pulsarApi.proto file is generated based on the older version. With the iteration of the version, the protocol file has undergone major changes. Here we need to try to be consistent with the proto file of the pulsar repo ### Modifications - Update `proto` file --- pulsar/internal/pulsar_proto/PulsarApi.pb.go | 2980 ++++++++++++------ pulsar/internal/pulsar_proto/PulsarApi.proto | 79 +- 2 files changed, 2161 insertions(+), 898 deletions(-) diff --git a/pulsar/internal/pulsar_proto/PulsarApi.pb.go b/pulsar/internal/pulsar_proto/PulsarApi.pb.go index 040faa997c..c5cefe75bb 100644 --- a/pulsar/internal/pulsar_proto/PulsarApi.pb.go +++ b/pulsar/internal/pulsar_proto/PulsarApi.pb.go @@ -1,5 +1,5 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: github.com/apache/pulsar-client-go@v0.3.0/pulsar/internal/pulsar_proto/PulsarApi.proto +// source: PulsarApi.proto package pulsar_proto @@ -70,7 +70,50 @@ func (x *CompressionType) UnmarshalJSON(data []byte) error { } func (CompressionType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{0} + return fileDescriptor_39529ba7ad9caeb8, []int{0} +} + +type ProducerAccessMode int32 + +const ( + ProducerAccessMode_Shared ProducerAccessMode = 0 + ProducerAccessMode_Exclusive ProducerAccessMode = 1 + ProducerAccessMode_WaitForExclusive ProducerAccessMode = 2 +) + +var ProducerAccessMode_name = map[int32]string{ + 0: "Shared", + 1: "Exclusive", + 2: "WaitForExclusive", +} + +var ProducerAccessMode_value = map[string]int32{ + "Shared": 0, + "Exclusive": 1, + "WaitForExclusive": 2, +} + +func (x ProducerAccessMode) Enum() *ProducerAccessMode { + p := new(ProducerAccessMode) + *p = x + return p +} + +func (x ProducerAccessMode) String() string { + return proto.EnumName(ProducerAccessMode_name, int32(x)) +} + +func (x *ProducerAccessMode) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(ProducerAccessMode_value, data, "ProducerAccessMode") + if err != nil { + return err + } + *x = ProducerAccessMode(value) + return nil +} + +func (ProducerAccessMode) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_39529ba7ad9caeb8, []int{1} } type ServerError int32 @@ -99,6 +142,10 @@ const ( ServerError_ConsumerAssignError ServerError = 19 ServerError_TransactionCoordinatorNotFound ServerError = 20 ServerError_InvalidTxnStatus ServerError = 21 + ServerError_NotAllowedError ServerError = 22 + ServerError_TransactionConflict ServerError = 23 + ServerError_TransactionNotFound ServerError = 24 + ServerError_ProducerFenced ServerError = 25 ) var ServerError_name = map[int32]string{ @@ -124,6 +171,10 @@ var ServerError_name = map[int32]string{ 19: "ConsumerAssignError", 20: "TransactionCoordinatorNotFound", 21: "InvalidTxnStatus", + 22: "NotAllowedError", + 23: "TransactionConflict", + 24: "TransactionNotFound", + 25: "ProducerFenced", } var ServerError_value = map[string]int32{ @@ -149,6 +200,10 @@ var ServerError_value = map[string]int32{ "ConsumerAssignError": 19, "TransactionCoordinatorNotFound": 20, "InvalidTxnStatus": 21, + "NotAllowedError": 22, + "TransactionConflict": 23, + "TransactionNotFound": 24, + "ProducerFenced": 25, } func (x ServerError) Enum() *ServerError { @@ -171,7 +226,7 @@ func (x *ServerError) UnmarshalJSON(data []byte) error { } func (ServerError) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{1} + return fileDescriptor_39529ba7ad9caeb8, []int{2} } type AuthMethod int32 @@ -214,7 +269,7 @@ func (x *AuthMethod) UnmarshalJSON(data []byte) error { } func (AuthMethod) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{2} + return fileDescriptor_39529ba7ad9caeb8, []int{3} } // Each protocol version identify new features that are @@ -241,6 +296,8 @@ const ( ProtocolVersion_v14 ProtocolVersion = 14 // Added Key_Shared subscription ProtocolVersion_v15 ProtocolVersion = 15 + ProtocolVersion_v16 ProtocolVersion = 16 + ProtocolVersion_v17 ProtocolVersion = 17 ) var ProtocolVersion_name = map[int32]string{ @@ -260,6 +317,8 @@ var ProtocolVersion_name = map[int32]string{ 13: "v13", 14: "v14", 15: "v15", + 16: "v16", + 17: "v17", } var ProtocolVersion_value = map[string]int32{ @@ -279,6 +338,8 @@ var ProtocolVersion_value = map[string]int32{ "v13": 13, "v14": 14, "v15": 15, + "v16": 16, + "v17": 17, } func (x ProtocolVersion) Enum() *ProtocolVersion { @@ -301,7 +362,7 @@ func (x *ProtocolVersion) UnmarshalJSON(data []byte) error { } func (ProtocolVersion) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{3} + return fileDescriptor_39529ba7ad9caeb8, []int{4} } type KeySharedMode int32 @@ -341,7 +402,7 @@ func (x *KeySharedMode) UnmarshalJSON(data []byte) error { } func (KeySharedMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{4} + return fileDescriptor_39529ba7ad9caeb8, []int{5} } type TxnAction int32 @@ -381,28 +442,33 @@ func (x *TxnAction) UnmarshalJSON(data []byte) error { } func (TxnAction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{5} + return fileDescriptor_39529ba7ad9caeb8, []int{6} } type Schema_Type int32 const ( - Schema_None Schema_Type = 0 - Schema_String Schema_Type = 1 - Schema_Json Schema_Type = 2 - Schema_Protobuf Schema_Type = 3 - Schema_Avro Schema_Type = 4 - Schema_Bool Schema_Type = 5 - Schema_Int8 Schema_Type = 6 - Schema_Int16 Schema_Type = 7 - Schema_Int32 Schema_Type = 8 - Schema_Int64 Schema_Type = 9 - Schema_Float Schema_Type = 10 - Schema_Double Schema_Type = 11 - Schema_Date Schema_Type = 12 - Schema_Time Schema_Type = 13 - Schema_Timestamp Schema_Type = 14 - Schema_KeyValue Schema_Type = 15 + Schema_None Schema_Type = 0 + Schema_String Schema_Type = 1 + Schema_Json Schema_Type = 2 + Schema_Protobuf Schema_Type = 3 + Schema_Avro Schema_Type = 4 + Schema_Bool Schema_Type = 5 + Schema_Int8 Schema_Type = 6 + Schema_Int16 Schema_Type = 7 + Schema_Int32 Schema_Type = 8 + Schema_Int64 Schema_Type = 9 + Schema_Float Schema_Type = 10 + Schema_Double Schema_Type = 11 + Schema_Date Schema_Type = 12 + Schema_Time Schema_Type = 13 + Schema_Timestamp Schema_Type = 14 + Schema_KeyValue Schema_Type = 15 + Schema_Instant Schema_Type = 16 + Schema_LocalDate Schema_Type = 17 + Schema_LocalTime Schema_Type = 18 + Schema_LocalDateTime Schema_Type = 19 + Schema_ProtobufNative Schema_Type = 20 ) var Schema_Type_name = map[int32]string{ @@ -422,25 +488,35 @@ var Schema_Type_name = map[int32]string{ 13: "Time", 14: "Timestamp", 15: "KeyValue", + 16: "Instant", + 17: "LocalDate", + 18: "LocalTime", + 19: "LocalDateTime", + 20: "ProtobufNative", } var Schema_Type_value = map[string]int32{ - "None": 0, - "String": 1, - "Json": 2, - "Protobuf": 3, - "Avro": 4, - "Bool": 5, - "Int8": 6, - "Int16": 7, - "Int32": 8, - "Int64": 9, - "Float": 10, - "Double": 11, - "Date": 12, - "Time": 13, - "Timestamp": 14, - "KeyValue": 15, + "None": 0, + "String": 1, + "Json": 2, + "Protobuf": 3, + "Avro": 4, + "Bool": 5, + "Int8": 6, + "Int16": 7, + "Int32": 8, + "Int64": 9, + "Float": 10, + "Double": 11, + "Date": 12, + "Time": 13, + "Timestamp": 14, + "KeyValue": 15, + "Instant": 16, + "LocalDate": 17, + "LocalTime": 18, + "LocalDateTime": 19, + "ProtobufNative": 20, } func (x Schema_Type) Enum() *Schema_Type { @@ -463,7 +539,7 @@ func (x *Schema_Type) UnmarshalJSON(data []byte) error { } func (Schema_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{0, 0} + return fileDescriptor_39529ba7ad9caeb8, []int{0, 0} } type CommandSubscribe_SubType int32 @@ -509,7 +585,7 @@ func (x *CommandSubscribe_SubType) UnmarshalJSON(data []byte) error { } func (CommandSubscribe_SubType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{15, 0} + return fileDescriptor_39529ba7ad9caeb8, []int{16, 0} } type CommandSubscribe_InitialPosition int32 @@ -549,7 +625,7 @@ func (x *CommandSubscribe_InitialPosition) UnmarshalJSON(data []byte) error { } func (CommandSubscribe_InitialPosition) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{15, 1} + return fileDescriptor_39529ba7ad9caeb8, []int{16, 1} } type CommandPartitionedTopicMetadataResponse_LookupType int32 @@ -589,7 +665,7 @@ func (x *CommandPartitionedTopicMetadataResponse_LookupType) UnmarshalJSON(data } func (CommandPartitionedTopicMetadataResponse_LookupType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{17, 0} + return fileDescriptor_39529ba7ad9caeb8, []int{18, 0} } type CommandLookupTopicResponse_LookupType int32 @@ -632,7 +708,7 @@ func (x *CommandLookupTopicResponse_LookupType) UnmarshalJSON(data []byte) error } func (CommandLookupTopicResponse_LookupType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{19, 0} + return fileDescriptor_39529ba7ad9caeb8, []int{20, 0} } type CommandAck_AckType int32 @@ -672,7 +748,7 @@ func (x *CommandAck_AckType) UnmarshalJSON(data []byte) error { } func (CommandAck_AckType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{25, 0} + return fileDescriptor_39529ba7ad9caeb8, []int{26, 0} } // Acks can contain a flag to indicate the consumer @@ -724,7 +800,7 @@ func (x *CommandAck_ValidationError) UnmarshalJSON(data []byte) error { } func (CommandAck_ValidationError) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{25, 1} + return fileDescriptor_39529ba7ad9caeb8, []int{26, 1} } type CommandGetTopicsOfNamespace_Mode int32 @@ -767,7 +843,7 @@ func (x *CommandGetTopicsOfNamespace_Mode) UnmarshalJSON(data []byte) error { } func (CommandGetTopicsOfNamespace_Mode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{44, 0} + return fileDescriptor_39529ba7ad9caeb8, []int{45, 0} } type BaseCommand_Type int32 @@ -955,12 +1031,12 @@ func (x *BaseCommand_Type) UnmarshalJSON(data []byte) error { } func (BaseCommand_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{63, 0} + return fileDescriptor_39529ba7ad9caeb8, []int{64, 0} } type Schema struct { Name *string `protobuf:"bytes,1,req,name=name" json:"name,omitempty"` - SchemaData []byte `protobuf:"bytes,3,req,name=schema_data" json:"schema_data,omitempty"` + SchemaData []byte `protobuf:"bytes,3,req,name=schema_data,json=schemaData" json:"schema_data,omitempty"` Type *Schema_Type `protobuf:"varint,4,req,name=type,enum=pulsar.proto.Schema_Type" json:"type,omitempty"` Properties []*KeyValue `protobuf:"bytes,5,rep,name=properties" json:"properties,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -972,7 +1048,7 @@ func (m *Schema) Reset() { *m = Schema{} } func (m *Schema) String() string { return proto.CompactTextString(m) } func (*Schema) ProtoMessage() {} func (*Schema) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{0} + return fileDescriptor_39529ba7ad9caeb8, []int{0} } func (m *Schema) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1033,7 +1109,9 @@ type MessageIdData struct { LedgerId *uint64 `protobuf:"varint,1,req,name=ledgerId" json:"ledgerId,omitempty"` EntryId *uint64 `protobuf:"varint,2,req,name=entryId" json:"entryId,omitempty"` Partition *int32 `protobuf:"varint,3,opt,name=partition,def=-1" json:"partition,omitempty"` - BatchIndex *int32 `protobuf:"varint,4,opt,name=batch_index,def=-1" json:"batch_index,omitempty"` + BatchIndex *int32 `protobuf:"varint,4,opt,name=batch_index,json=batchIndex,def=-1" json:"batch_index,omitempty"` + AckSet []int64 `protobuf:"varint,5,rep,name=ack_set,json=ackSet" json:"ack_set,omitempty"` + BatchSize *int32 `protobuf:"varint,6,opt,name=batch_size,json=batchSize" json:"batch_size,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1043,7 +1121,7 @@ func (m *MessageIdData) Reset() { *m = MessageIdData{} } func (m *MessageIdData) String() string { return proto.CompactTextString(m) } func (*MessageIdData) ProtoMessage() {} func (*MessageIdData) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{1} + return fileDescriptor_39529ba7ad9caeb8, []int{1} } func (m *MessageIdData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1103,6 +1181,20 @@ func (m *MessageIdData) GetBatchIndex() int32 { return Default_MessageIdData_BatchIndex } +func (m *MessageIdData) GetAckSet() []int64 { + if m != nil { + return m.AckSet + } + return nil +} + +func (m *MessageIdData) GetBatchSize() int32 { + if m != nil && m.BatchSize != nil { + return *m.BatchSize + } + return 0 +} + type KeyValue struct { Key *string `protobuf:"bytes,1,req,name=key" json:"key,omitempty"` Value *string `protobuf:"bytes,2,req,name=value" json:"value,omitempty"` @@ -1115,7 +1207,7 @@ func (m *KeyValue) Reset() { *m = KeyValue{} } func (m *KeyValue) String() string { return proto.CompactTextString(m) } func (*KeyValue) ProtoMessage() {} func (*KeyValue) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{2} + return fileDescriptor_39529ba7ad9caeb8, []int{2} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1170,7 +1262,7 @@ func (m *KeyLongValue) Reset() { *m = KeyLongValue{} } func (m *KeyLongValue) String() string { return proto.CompactTextString(m) } func (*KeyLongValue) ProtoMessage() {} func (*KeyLongValue) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{3} + return fileDescriptor_39529ba7ad9caeb8, []int{3} } func (m *KeyLongValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1225,7 +1317,7 @@ func (m *IntRange) Reset() { *m = IntRange{} } func (m *IntRange) String() string { return proto.CompactTextString(m) } func (*IntRange) ProtoMessage() {} func (*IntRange) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{4} + return fileDescriptor_39529ba7ad9caeb8, []int{4} } func (m *IntRange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1281,7 +1373,7 @@ func (m *EncryptionKeys) Reset() { *m = EncryptionKeys{} } func (m *EncryptionKeys) String() string { return proto.CompactTextString(m) } func (*EncryptionKeys) ProtoMessage() {} func (*EncryptionKeys) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{5} + return fileDescriptor_39529ba7ad9caeb8, []int{5} } func (m *EncryptionKeys) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1332,48 +1424,56 @@ func (m *EncryptionKeys) GetMetadata() []*KeyValue { } type MessageMetadata struct { - ProducerName *string `protobuf:"bytes,1,req,name=producer_name" json:"producer_name,omitempty"` - SequenceId *uint64 `protobuf:"varint,2,req,name=sequence_id" json:"sequence_id,omitempty"` - PublishTime *uint64 `protobuf:"varint,3,req,name=publish_time" json:"publish_time,omitempty"` + ProducerName *string `protobuf:"bytes,1,req,name=producer_name,json=producerName" json:"producer_name,omitempty"` + SequenceId *uint64 `protobuf:"varint,2,req,name=sequence_id,json=sequenceId" json:"sequence_id,omitempty"` + PublishTime *uint64 `protobuf:"varint,3,req,name=publish_time,json=publishTime" json:"publish_time,omitempty"` Properties []*KeyValue `protobuf:"bytes,4,rep,name=properties" json:"properties,omitempty"` // Property set on replicated message, // includes the source cluster name - ReplicatedFrom *string `protobuf:"bytes,5,opt,name=replicated_from" json:"replicated_from,omitempty"` + ReplicatedFrom *string `protobuf:"bytes,5,opt,name=replicated_from,json=replicatedFrom" json:"replicated_from,omitempty"` //key to decide partition for the msg - PartitionKey *string `protobuf:"bytes,6,opt,name=partition_key" json:"partition_key,omitempty"` + PartitionKey *string `protobuf:"bytes,6,opt,name=partition_key,json=partitionKey" json:"partition_key,omitempty"` // Override namespace's replication - ReplicateTo []string `protobuf:"bytes,7,rep,name=replicate_to" json:"replicate_to,omitempty"` + ReplicateTo []string `protobuf:"bytes,7,rep,name=replicate_to,json=replicateTo" json:"replicate_to,omitempty"` Compression *CompressionType `protobuf:"varint,8,opt,name=compression,enum=pulsar.proto.CompressionType,def=0" json:"compression,omitempty"` - UncompressedSize *uint32 `protobuf:"varint,9,opt,name=uncompressed_size,def=0" json:"uncompressed_size,omitempty"` + UncompressedSize *uint32 `protobuf:"varint,9,opt,name=uncompressed_size,json=uncompressedSize,def=0" json:"uncompressed_size,omitempty"` // Removed below checksum field from Metadata as // it should be part of send-command which keeps checksum of header + payload //optional sfixed64 checksum = 10; // differentiate single and batch message metadata - NumMessagesInBatch *int32 `protobuf:"varint,11,opt,name=num_messages_in_batch,def=1" json:"num_messages_in_batch,omitempty"` + NumMessagesInBatch *int32 `protobuf:"varint,11,opt,name=num_messages_in_batch,json=numMessagesInBatch,def=1" json:"num_messages_in_batch,omitempty"` // the timestamp that this event occurs. it is typically set by applications. // if this field is omitted, `publish_time` can be used for the purpose of `event_time`. - EventTime *uint64 `protobuf:"varint,12,opt,name=event_time,def=0" json:"event_time,omitempty"` + EventTime *uint64 `protobuf:"varint,12,opt,name=event_time,json=eventTime,def=0" json:"event_time,omitempty"` // Contains encryption key name, encrypted key and metadata to describe the key - EncryptionKeys []*EncryptionKeys `protobuf:"bytes,13,rep,name=encryption_keys" json:"encryption_keys,omitempty"` + EncryptionKeys []*EncryptionKeys `protobuf:"bytes,13,rep,name=encryption_keys,json=encryptionKeys" json:"encryption_keys,omitempty"` // Algorithm used to encrypt data key - EncryptionAlgo *string `protobuf:"bytes,14,opt,name=encryption_algo" json:"encryption_algo,omitempty"` + EncryptionAlgo *string `protobuf:"bytes,14,opt,name=encryption_algo,json=encryptionAlgo" json:"encryption_algo,omitempty"` // Additional parameters required by encryption - EncryptionParam []byte `protobuf:"bytes,15,opt,name=encryption_param" json:"encryption_param,omitempty"` - SchemaVersion []byte `protobuf:"bytes,16,opt,name=schema_version" json:"schema_version,omitempty"` - PartitionKeyB64Encoded *bool `protobuf:"varint,17,opt,name=partition_key_b64_encoded,def=0" json:"partition_key_b64_encoded,omitempty"` + EncryptionParam []byte `protobuf:"bytes,15,opt,name=encryption_param,json=encryptionParam" json:"encryption_param,omitempty"` + SchemaVersion []byte `protobuf:"bytes,16,opt,name=schema_version,json=schemaVersion" json:"schema_version,omitempty"` + PartitionKeyB64Encoded *bool `protobuf:"varint,17,opt,name=partition_key_b64_encoded,json=partitionKeyB64Encoded,def=0" json:"partition_key_b64_encoded,omitempty"` // Specific a key to overwrite the message key which used for ordering dispatch in Key_Shared mode. - OrderingKey []byte `protobuf:"bytes,18,opt,name=ordering_key" json:"ordering_key,omitempty"` + OrderingKey []byte `protobuf:"bytes,18,opt,name=ordering_key,json=orderingKey" json:"ordering_key,omitempty"` // Mark the message to be delivered at or after the specified timestamp - DeliverAtTime *int64 `protobuf:"varint,19,opt,name=deliver_at_time" json:"deliver_at_time,omitempty"` + DeliverAtTime *int64 `protobuf:"varint,19,opt,name=deliver_at_time,json=deliverAtTime" json:"deliver_at_time,omitempty"` // Identify whether a message is a "marker" message used for // internal metadata instead of application published data. // Markers will generally not be propagated back to clients - MarkerType *int32 `protobuf:"varint,20,opt,name=marker_type" json:"marker_type,omitempty"` + MarkerType *int32 `protobuf:"varint,20,opt,name=marker_type,json=markerType" json:"marker_type,omitempty"` // transaction related message info - TxnidLeastBits *uint64 `protobuf:"varint,22,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,23,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,22,opt,name=txnid_least_bits,json=txnidLeastBits" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,23,opt,name=txnid_most_bits,json=txnidMostBits" json:"txnid_most_bits,omitempty"` /// Add highest sequence id to support batch message with external sequence id - HighestSequenceId *uint64 `protobuf:"varint,24,opt,name=highest_sequence_id,def=0" json:"highest_sequence_id,omitempty"` + HighestSequenceId *uint64 `protobuf:"varint,24,opt,name=highest_sequence_id,json=highestSequenceId,def=0" json:"highest_sequence_id,omitempty"` + // Indicate if the message payload value is set + NullValue *bool `protobuf:"varint,25,opt,name=null_value,json=nullValue,def=0" json:"null_value,omitempty"` + Uuid *string `protobuf:"bytes,26,opt,name=uuid" json:"uuid,omitempty"` + NumChunksFromMsg *int32 `protobuf:"varint,27,opt,name=num_chunks_from_msg,json=numChunksFromMsg" json:"num_chunks_from_msg,omitempty"` + TotalChunkMsgSize *int32 `protobuf:"varint,28,opt,name=total_chunk_msg_size,json=totalChunkMsgSize" json:"total_chunk_msg_size,omitempty"` + ChunkId *int32 `protobuf:"varint,29,opt,name=chunk_id,json=chunkId" json:"chunk_id,omitempty"` + // Indicate if the message partition key is set + NullPartitionKey *bool `protobuf:"varint,30,opt,name=null_partition_key,json=nullPartitionKey,def=0" json:"null_partition_key,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1383,7 +1483,7 @@ func (m *MessageMetadata) Reset() { *m = MessageMetadata{} } func (m *MessageMetadata) String() string { return proto.CompactTextString(m) } func (*MessageMetadata) ProtoMessage() {} func (*MessageMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{6} + return fileDescriptor_39529ba7ad9caeb8, []int{6} } func (m *MessageMetadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1417,9 +1517,9 @@ const Default_MessageMetadata_UncompressedSize uint32 = 0 const Default_MessageMetadata_NumMessagesInBatch int32 = 1 const Default_MessageMetadata_EventTime uint64 = 0 const Default_MessageMetadata_PartitionKeyB64Encoded bool = false -const Default_MessageMetadata_TxnidLeastBits uint64 = 0 -const Default_MessageMetadata_TxnidMostBits uint64 = 0 const Default_MessageMetadata_HighestSequenceId uint64 = 0 +const Default_MessageMetadata_NullValue bool = false +const Default_MessageMetadata_NullPartitionKey bool = false func (m *MessageMetadata) GetProducerName() string { if m != nil && m.ProducerName != nil { @@ -1558,14 +1658,14 @@ func (m *MessageMetadata) GetTxnidLeastBits() uint64 { if m != nil && m.TxnidLeastBits != nil { return *m.TxnidLeastBits } - return Default_MessageMetadata_TxnidLeastBits + return 0 } func (m *MessageMetadata) GetTxnidMostBits() uint64 { if m != nil && m.TxnidMostBits != nil { return *m.TxnidMostBits } - return Default_MessageMetadata_TxnidMostBits + return 0 } func (m *MessageMetadata) GetHighestSequenceId() uint64 { @@ -1575,19 +1675,65 @@ func (m *MessageMetadata) GetHighestSequenceId() uint64 { return Default_MessageMetadata_HighestSequenceId } +func (m *MessageMetadata) GetNullValue() bool { + if m != nil && m.NullValue != nil { + return *m.NullValue + } + return Default_MessageMetadata_NullValue +} + +func (m *MessageMetadata) GetUuid() string { + if m != nil && m.Uuid != nil { + return *m.Uuid + } + return "" +} + +func (m *MessageMetadata) GetNumChunksFromMsg() int32 { + if m != nil && m.NumChunksFromMsg != nil { + return *m.NumChunksFromMsg + } + return 0 +} + +func (m *MessageMetadata) GetTotalChunkMsgSize() int32 { + if m != nil && m.TotalChunkMsgSize != nil { + return *m.TotalChunkMsgSize + } + return 0 +} + +func (m *MessageMetadata) GetChunkId() int32 { + if m != nil && m.ChunkId != nil { + return *m.ChunkId + } + return 0 +} + +func (m *MessageMetadata) GetNullPartitionKey() bool { + if m != nil && m.NullPartitionKey != nil { + return *m.NullPartitionKey + } + return Default_MessageMetadata_NullPartitionKey +} + type SingleMessageMetadata struct { Properties []*KeyValue `protobuf:"bytes,1,rep,name=properties" json:"properties,omitempty"` - PartitionKey *string `protobuf:"bytes,2,opt,name=partition_key" json:"partition_key,omitempty"` - PayloadSize *int32 `protobuf:"varint,3,req,name=payload_size" json:"payload_size,omitempty"` - CompactedOut *bool `protobuf:"varint,4,opt,name=compacted_out,def=0" json:"compacted_out,omitempty"` + PartitionKey *string `protobuf:"bytes,2,opt,name=partition_key,json=partitionKey" json:"partition_key,omitempty"` + PayloadSize *int32 `protobuf:"varint,3,req,name=payload_size,json=payloadSize" json:"payload_size,omitempty"` + CompactedOut *bool `protobuf:"varint,4,opt,name=compacted_out,json=compactedOut,def=0" json:"compacted_out,omitempty"` // the timestamp that this event occurs. it is typically set by applications. // if this field is omitted, `publish_time` can be used for the purpose of `event_time`. - EventTime *uint64 `protobuf:"varint,5,opt,name=event_time,def=0" json:"event_time,omitempty"` - PartitionKeyB64Encoded *bool `protobuf:"varint,6,opt,name=partition_key_b64_encoded,def=0" json:"partition_key_b64_encoded,omitempty"` + EventTime *uint64 `protobuf:"varint,5,opt,name=event_time,json=eventTime,def=0" json:"event_time,omitempty"` + PartitionKeyB64Encoded *bool `protobuf:"varint,6,opt,name=partition_key_b64_encoded,json=partitionKeyB64Encoded,def=0" json:"partition_key_b64_encoded,omitempty"` // Specific a key to overwrite the message key which used for ordering dispatch in Key_Shared mode. - OrderingKey []byte `protobuf:"bytes,7,opt,name=ordering_key" json:"ordering_key,omitempty"` + OrderingKey []byte `protobuf:"bytes,7,opt,name=ordering_key,json=orderingKey" json:"ordering_key,omitempty"` // Allows consumer retrieve the sequence id that the producer set. - SequenceId *uint64 `protobuf:"varint,8,opt,name=sequence_id" json:"sequence_id,omitempty"` + SequenceId *uint64 `protobuf:"varint,8,opt,name=sequence_id,json=sequenceId" json:"sequence_id,omitempty"` + // Indicate if the message payload value is set + NullValue *bool `protobuf:"varint,9,opt,name=null_value,json=nullValue,def=0" json:"null_value,omitempty"` + // Indicate if the message partition key is set + NullPartitionKey *bool `protobuf:"varint,10,opt,name=null_partition_key,json=nullPartitionKey,def=0" json:"null_partition_key,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1597,7 +1743,7 @@ func (m *SingleMessageMetadata) Reset() { *m = SingleMessageMetadata{} } func (m *SingleMessageMetadata) String() string { return proto.CompactTextString(m) } func (*SingleMessageMetadata) ProtoMessage() {} func (*SingleMessageMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{7} + return fileDescriptor_39529ba7ad9caeb8, []int{7} } func (m *SingleMessageMetadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1629,6 +1775,8 @@ var xxx_messageInfo_SingleMessageMetadata proto.InternalMessageInfo const Default_SingleMessageMetadata_CompactedOut bool = false const Default_SingleMessageMetadata_EventTime uint64 = 0 const Default_SingleMessageMetadata_PartitionKeyB64Encoded bool = false +const Default_SingleMessageMetadata_NullValue bool = false +const Default_SingleMessageMetadata_NullPartitionKey bool = false func (m *SingleMessageMetadata) GetProperties() []*KeyValue { if m != nil { @@ -1686,26 +1834,96 @@ func (m *SingleMessageMetadata) GetSequenceId() uint64 { return 0 } +func (m *SingleMessageMetadata) GetNullValue() bool { + if m != nil && m.NullValue != nil { + return *m.NullValue + } + return Default_SingleMessageMetadata_NullValue +} + +func (m *SingleMessageMetadata) GetNullPartitionKey() bool { + if m != nil && m.NullPartitionKey != nil { + return *m.NullPartitionKey + } + return Default_SingleMessageMetadata_NullPartitionKey +} + +// metadata added for entry from broker +type BrokerEntryMetadata struct { + BrokerTimestamp *uint64 `protobuf:"varint,1,opt,name=broker_timestamp,json=brokerTimestamp" json:"broker_timestamp,omitempty"` + Index *uint64 `protobuf:"varint,2,opt,name=index" json:"index,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BrokerEntryMetadata) Reset() { *m = BrokerEntryMetadata{} } +func (m *BrokerEntryMetadata) String() string { return proto.CompactTextString(m) } +func (*BrokerEntryMetadata) ProtoMessage() {} +func (*BrokerEntryMetadata) Descriptor() ([]byte, []int) { + return fileDescriptor_39529ba7ad9caeb8, []int{8} +} +func (m *BrokerEntryMetadata) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BrokerEntryMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_BrokerEntryMetadata.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *BrokerEntryMetadata) XXX_Merge(src proto.Message) { + xxx_messageInfo_BrokerEntryMetadata.Merge(m, src) +} +func (m *BrokerEntryMetadata) XXX_Size() int { + return m.Size() +} +func (m *BrokerEntryMetadata) XXX_DiscardUnknown() { + xxx_messageInfo_BrokerEntryMetadata.DiscardUnknown(m) +} + +var xxx_messageInfo_BrokerEntryMetadata proto.InternalMessageInfo + +func (m *BrokerEntryMetadata) GetBrokerTimestamp() uint64 { + if m != nil && m.BrokerTimestamp != nil { + return *m.BrokerTimestamp + } + return 0 +} + +func (m *BrokerEntryMetadata) GetIndex() uint64 { + if m != nil && m.Index != nil { + return *m.Index + } + return 0 +} + type CommandConnect struct { - ClientVersion *string `protobuf:"bytes,1,req,name=client_version" json:"client_version,omitempty"` - AuthMethod *AuthMethod `protobuf:"varint,2,opt,name=auth_method,enum=pulsar.proto.AuthMethod" json:"auth_method,omitempty"` - AuthMethodName *string `protobuf:"bytes,5,opt,name=auth_method_name" json:"auth_method_name,omitempty"` - AuthData []byte `protobuf:"bytes,3,opt,name=auth_data" json:"auth_data,omitempty"` - ProtocolVersion *int32 `protobuf:"varint,4,opt,name=protocol_version,def=0" json:"protocol_version,omitempty"` + ClientVersion *string `protobuf:"bytes,1,req,name=client_version,json=clientVersion" json:"client_version,omitempty"` + AuthMethod *AuthMethod `protobuf:"varint,2,opt,name=auth_method,json=authMethod,enum=pulsar.proto.AuthMethod" json:"auth_method,omitempty"` + AuthMethodName *string `protobuf:"bytes,5,opt,name=auth_method_name,json=authMethodName" json:"auth_method_name,omitempty"` + AuthData []byte `protobuf:"bytes,3,opt,name=auth_data,json=authData" json:"auth_data,omitempty"` + ProtocolVersion *int32 `protobuf:"varint,4,opt,name=protocol_version,json=protocolVersion,def=0" json:"protocol_version,omitempty"` // Client can ask to be proxyied to a specific broker // This is only honored by a Pulsar proxy - ProxyToBrokerUrl *string `protobuf:"bytes,6,opt,name=proxy_to_broker_url" json:"proxy_to_broker_url,omitempty"` + ProxyToBrokerUrl *string `protobuf:"bytes,6,opt,name=proxy_to_broker_url,json=proxyToBrokerUrl" json:"proxy_to_broker_url,omitempty"` // Original principal that was verified by // a Pulsar proxy. In this case the auth info above // will be the auth of the proxy itself - OriginalPrincipal *string `protobuf:"bytes,7,opt,name=original_principal" json:"original_principal,omitempty"` + OriginalPrincipal *string `protobuf:"bytes,7,opt,name=original_principal,json=originalPrincipal" json:"original_principal,omitempty"` // Original auth role and auth Method that was passed // to the proxy. In this case the auth info above // will be the auth of the proxy itself - OriginalAuthData *string `protobuf:"bytes,8,opt,name=original_auth_data" json:"original_auth_data,omitempty"` - OriginalAuthMethod *string `protobuf:"bytes,9,opt,name=original_auth_method" json:"original_auth_method,omitempty"` + OriginalAuthData *string `protobuf:"bytes,8,opt,name=original_auth_data,json=originalAuthData" json:"original_auth_data,omitempty"` + OriginalAuthMethod *string `protobuf:"bytes,9,opt,name=original_auth_method,json=originalAuthMethod" json:"original_auth_method,omitempty"` // Feature flags - FeatureFlags *FeatureFlags `protobuf:"bytes,10,opt,name=feature_flags" json:"feature_flags,omitempty"` + FeatureFlags *FeatureFlags `protobuf:"bytes,10,opt,name=feature_flags,json=featureFlags" json:"feature_flags,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1715,7 +1933,7 @@ func (m *CommandConnect) Reset() { *m = CommandConnect{} } func (m *CommandConnect) String() string { return proto.CompactTextString(m) } func (*CommandConnect) ProtoMessage() {} func (*CommandConnect) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{8} + return fileDescriptor_39529ba7ad9caeb8, []int{9} } func (m *CommandConnect) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1817,17 +2035,18 @@ func (m *CommandConnect) GetFeatureFlags() *FeatureFlags { } type FeatureFlags struct { - SupportsAuthRefresh *bool `protobuf:"varint,1,opt,name=supports_auth_refresh,def=0" json:"supports_auth_refresh,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + SupportsAuthRefresh *bool `protobuf:"varint,1,opt,name=supports_auth_refresh,json=supportsAuthRefresh,def=0" json:"supports_auth_refresh,omitempty"` + SupportsBrokerEntryMetadata *bool `protobuf:"varint,2,opt,name=supports_broker_entry_metadata,json=supportsBrokerEntryMetadata,def=0" json:"supports_broker_entry_metadata,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *FeatureFlags) Reset() { *m = FeatureFlags{} } func (m *FeatureFlags) String() string { return proto.CompactTextString(m) } func (*FeatureFlags) ProtoMessage() {} func (*FeatureFlags) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{9} + return fileDescriptor_39529ba7ad9caeb8, []int{10} } func (m *FeatureFlags) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1857,6 +2076,7 @@ func (m *FeatureFlags) XXX_DiscardUnknown() { var xxx_messageInfo_FeatureFlags proto.InternalMessageInfo const Default_FeatureFlags_SupportsAuthRefresh bool = false +const Default_FeatureFlags_SupportsBrokerEntryMetadata bool = false func (m *FeatureFlags) GetSupportsAuthRefresh() bool { if m != nil && m.SupportsAuthRefresh != nil { @@ -1865,10 +2085,17 @@ func (m *FeatureFlags) GetSupportsAuthRefresh() bool { return Default_FeatureFlags_SupportsAuthRefresh } +func (m *FeatureFlags) GetSupportsBrokerEntryMetadata() bool { + if m != nil && m.SupportsBrokerEntryMetadata != nil { + return *m.SupportsBrokerEntryMetadata + } + return Default_FeatureFlags_SupportsBrokerEntryMetadata +} + type CommandConnected struct { - ServerVersion *string `protobuf:"bytes,1,req,name=server_version" json:"server_version,omitempty"` - ProtocolVersion *int32 `protobuf:"varint,2,opt,name=protocol_version,def=0" json:"protocol_version,omitempty"` - MaxMessageSize *int32 `protobuf:"varint,3,opt,name=max_message_size" json:"max_message_size,omitempty"` + ServerVersion *string `protobuf:"bytes,1,req,name=server_version,json=serverVersion" json:"server_version,omitempty"` + ProtocolVersion *int32 `protobuf:"varint,2,opt,name=protocol_version,json=protocolVersion,def=0" json:"protocol_version,omitempty"` + MaxMessageSize *int32 `protobuf:"varint,3,opt,name=max_message_size,json=maxMessageSize" json:"max_message_size,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1878,7 +2105,7 @@ func (m *CommandConnected) Reset() { *m = CommandConnected{} } func (m *CommandConnected) String() string { return proto.CompactTextString(m) } func (*CommandConnected) ProtoMessage() {} func (*CommandConnected) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{10} + return fileDescriptor_39529ba7ad9caeb8, []int{11} } func (m *CommandConnected) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1931,9 +2158,9 @@ func (m *CommandConnected) GetMaxMessageSize() int32 { } type CommandAuthResponse struct { - ClientVersion *string `protobuf:"bytes,1,opt,name=client_version" json:"client_version,omitempty"` + ClientVersion *string `protobuf:"bytes,1,opt,name=client_version,json=clientVersion" json:"client_version,omitempty"` Response *AuthData `protobuf:"bytes,2,opt,name=response" json:"response,omitempty"` - ProtocolVersion *int32 `protobuf:"varint,3,opt,name=protocol_version,def=0" json:"protocol_version,omitempty"` + ProtocolVersion *int32 `protobuf:"varint,3,opt,name=protocol_version,json=protocolVersion,def=0" json:"protocol_version,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1943,7 +2170,7 @@ func (m *CommandAuthResponse) Reset() { *m = CommandAuthResponse{} } func (m *CommandAuthResponse) String() string { return proto.CompactTextString(m) } func (*CommandAuthResponse) ProtoMessage() {} func (*CommandAuthResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{11} + return fileDescriptor_39529ba7ad9caeb8, []int{12} } func (m *CommandAuthResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1996,9 +2223,9 @@ func (m *CommandAuthResponse) GetProtocolVersion() int32 { } type CommandAuthChallenge struct { - ServerVersion *string `protobuf:"bytes,1,opt,name=server_version" json:"server_version,omitempty"` + ServerVersion *string `protobuf:"bytes,1,opt,name=server_version,json=serverVersion" json:"server_version,omitempty"` Challenge *AuthData `protobuf:"bytes,2,opt,name=challenge" json:"challenge,omitempty"` - ProtocolVersion *int32 `protobuf:"varint,3,opt,name=protocol_version,def=0" json:"protocol_version,omitempty"` + ProtocolVersion *int32 `protobuf:"varint,3,opt,name=protocol_version,json=protocolVersion,def=0" json:"protocol_version,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2008,7 +2235,7 @@ func (m *CommandAuthChallenge) Reset() { *m = CommandAuthChallenge{} } func (m *CommandAuthChallenge) String() string { return proto.CompactTextString(m) } func (*CommandAuthChallenge) ProtoMessage() {} func (*CommandAuthChallenge) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{12} + return fileDescriptor_39529ba7ad9caeb8, []int{13} } func (m *CommandAuthChallenge) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2062,8 +2289,8 @@ func (m *CommandAuthChallenge) GetProtocolVersion() int32 { // To support mutual authentication type, such as Sasl, reuse this command to mutual auth. type AuthData struct { - AuthMethodName *string `protobuf:"bytes,1,opt,name=auth_method_name" json:"auth_method_name,omitempty"` - AuthData []byte `protobuf:"bytes,2,opt,name=auth_data" json:"auth_data,omitempty"` + AuthMethodName *string `protobuf:"bytes,1,opt,name=auth_method_name,json=authMethodName" json:"auth_method_name,omitempty"` + AuthData []byte `protobuf:"bytes,2,opt,name=auth_data,json=authData" json:"auth_data,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2073,7 +2300,7 @@ func (m *AuthData) Reset() { *m = AuthData{} } func (m *AuthData) String() string { return proto.CompactTextString(m) } func (*AuthData) ProtoMessage() {} func (*AuthData) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{13} + return fileDescriptor_39529ba7ad9caeb8, []int{14} } func (m *AuthData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2129,7 +2356,7 @@ func (m *KeySharedMeta) Reset() { *m = KeySharedMeta{} } func (m *KeySharedMeta) String() string { return proto.CompactTextString(m) } func (*KeySharedMeta) ProtoMessage() {} func (*KeySharedMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{14} + return fileDescriptor_39529ba7ad9caeb8, []int{15} } func (m *KeySharedMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2185,20 +2412,20 @@ type CommandSubscribe struct { Topic *string `protobuf:"bytes,1,req,name=topic" json:"topic,omitempty"` Subscription *string `protobuf:"bytes,2,req,name=subscription" json:"subscription,omitempty"` SubType *CommandSubscribe_SubType `protobuf:"varint,3,req,name=subType,enum=pulsar.proto.CommandSubscribe_SubType" json:"subType,omitempty"` - ConsumerId *uint64 `protobuf:"varint,4,req,name=consumer_id" json:"consumer_id,omitempty"` - RequestId *uint64 `protobuf:"varint,5,req,name=request_id" json:"request_id,omitempty"` - ConsumerName *string `protobuf:"bytes,6,opt,name=consumer_name" json:"consumer_name,omitempty"` - PriorityLevel *int32 `protobuf:"varint,7,opt,name=priority_level" json:"priority_level,omitempty"` + ConsumerId *uint64 `protobuf:"varint,4,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` + RequestId *uint64 `protobuf:"varint,5,req,name=request_id,json=requestId" json:"request_id,omitempty"` + ConsumerName *string `protobuf:"bytes,6,opt,name=consumer_name,json=consumerName" json:"consumer_name,omitempty"` + PriorityLevel *int32 `protobuf:"varint,7,opt,name=priority_level,json=priorityLevel" json:"priority_level,omitempty"` // Signal wether the subscription should be backed by a // durable cursor or not Durable *bool `protobuf:"varint,8,opt,name=durable,def=1" json:"durable,omitempty"` // If specified, the subscription will position the cursor // markd-delete position on the particular message id and // will send messages from that point - StartMessageId *MessageIdData `protobuf:"bytes,9,opt,name=start_message_id" json:"start_message_id,omitempty"` + StartMessageId *MessageIdData `protobuf:"bytes,9,opt,name=start_message_id,json=startMessageId" json:"start_message_id,omitempty"` /// Add optional metadata key=value to this consumer Metadata []*KeyValue `protobuf:"bytes,10,rep,name=metadata" json:"metadata,omitempty"` - ReadCompacted *bool `protobuf:"varint,11,opt,name=read_compacted" json:"read_compacted,omitempty"` + ReadCompacted *bool `protobuf:"varint,11,opt,name=read_compacted,json=readCompacted" json:"read_compacted,omitempty"` Schema *Schema `protobuf:"bytes,12,opt,name=schema" json:"schema,omitempty"` // Signal whether the subscription will initialize on latest // or not -- earliest @@ -2206,16 +2433,16 @@ type CommandSubscribe struct { // Mark the subscription as "replicated". Pulsar will make sure // to periodically sync the state of replicated subscriptions // across different clusters (when using geo-replication). - ReplicateSubscriptionState *bool `protobuf:"varint,14,opt,name=replicate_subscription_state" json:"replicate_subscription_state,omitempty"` + ReplicateSubscriptionState *bool `protobuf:"varint,14,opt,name=replicate_subscription_state,json=replicateSubscriptionState" json:"replicate_subscription_state,omitempty"` // If true, the subscribe operation will cause a topic to be // created if it does not exist already (and if topic auto-creation // is allowed by broker. // If false, the subscribe operation will fail if the topic // does not exist. - ForceTopicCreation *bool `protobuf:"varint,15,opt,name=force_topic_creation,def=1" json:"force_topic_creation,omitempty"` + ForceTopicCreation *bool `protobuf:"varint,15,opt,name=force_topic_creation,json=forceTopicCreation,def=1" json:"force_topic_creation,omitempty"` // If specified, the subscription will reset cursor's position back // to specified seconds and will send messages from that point - StartMessageRollbackDurationSec *uint64 `protobuf:"varint,16,opt,name=start_message_rollback_duration_sec,def=0" json:"start_message_rollback_duration_sec,omitempty"` + StartMessageRollbackDurationSec *uint64 `protobuf:"varint,16,opt,name=start_message_rollback_duration_sec,json=startMessageRollbackDurationSec,def=0" json:"start_message_rollback_duration_sec,omitempty"` KeySharedMeta *KeySharedMeta `protobuf:"bytes,17,opt,name=keySharedMeta" json:"keySharedMeta,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -2226,7 +2453,7 @@ func (m *CommandSubscribe) Reset() { *m = CommandSubscribe{} } func (m *CommandSubscribe) String() string { return proto.CompactTextString(m) } func (*CommandSubscribe) ProtoMessage() {} func (*CommandSubscribe) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{15} + return fileDescriptor_39529ba7ad9caeb8, []int{16} } func (m *CommandSubscribe) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2381,15 +2608,15 @@ func (m *CommandSubscribe) GetKeySharedMeta() *KeySharedMeta { type CommandPartitionedTopicMetadata struct { Topic *string `protobuf:"bytes,1,req,name=topic" json:"topic,omitempty"` - RequestId *uint64 `protobuf:"varint,2,req,name=request_id" json:"request_id,omitempty"` + RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` // TODO - Remove original_principal, original_auth_data, original_auth_method // Original principal that was verified by // a Pulsar proxy. - OriginalPrincipal *string `protobuf:"bytes,3,opt,name=original_principal" json:"original_principal,omitempty"` + OriginalPrincipal *string `protobuf:"bytes,3,opt,name=original_principal,json=originalPrincipal" json:"original_principal,omitempty"` // Original auth role and auth Method that was passed // to the proxy. - OriginalAuthData *string `protobuf:"bytes,4,opt,name=original_auth_data" json:"original_auth_data,omitempty"` - OriginalAuthMethod *string `protobuf:"bytes,5,opt,name=original_auth_method" json:"original_auth_method,omitempty"` + OriginalAuthData *string `protobuf:"bytes,4,opt,name=original_auth_data,json=originalAuthData" json:"original_auth_data,omitempty"` + OriginalAuthMethod *string `protobuf:"bytes,5,opt,name=original_auth_method,json=originalAuthMethod" json:"original_auth_method,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2399,7 +2626,7 @@ func (m *CommandPartitionedTopicMetadata) Reset() { *m = CommandPartitio func (m *CommandPartitionedTopicMetadata) String() string { return proto.CompactTextString(m) } func (*CommandPartitionedTopicMetadata) ProtoMessage() {} func (*CommandPartitionedTopicMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{16} + return fileDescriptor_39529ba7ad9caeb8, []int{17} } func (m *CommandPartitionedTopicMetadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2465,7 +2692,7 @@ func (m *CommandPartitionedTopicMetadata) GetOriginalAuthMethod() string { type CommandPartitionedTopicMetadataResponse struct { Partitions *uint32 `protobuf:"varint,1,opt,name=partitions" json:"partitions,omitempty"` - RequestId *uint64 `protobuf:"varint,2,req,name=request_id" json:"request_id,omitempty"` + RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` Response *CommandPartitionedTopicMetadataResponse_LookupType `protobuf:"varint,3,opt,name=response,enum=pulsar.proto.CommandPartitionedTopicMetadataResponse_LookupType" json:"response,omitempty"` Error *ServerError `protobuf:"varint,4,opt,name=error,enum=pulsar.proto.ServerError" json:"error,omitempty"` Message *string `protobuf:"bytes,5,opt,name=message" json:"message,omitempty"` @@ -2480,7 +2707,7 @@ func (m *CommandPartitionedTopicMetadataResponse) Reset() { func (m *CommandPartitionedTopicMetadataResponse) String() string { return proto.CompactTextString(m) } func (*CommandPartitionedTopicMetadataResponse) ProtoMessage() {} func (*CommandPartitionedTopicMetadataResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{17} + return fileDescriptor_39529ba7ad9caeb8, []int{18} } func (m *CommandPartitionedTopicMetadataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2546,18 +2773,18 @@ func (m *CommandPartitionedTopicMetadataResponse) GetMessage() string { type CommandLookupTopic struct { Topic *string `protobuf:"bytes,1,req,name=topic" json:"topic,omitempty"` - RequestId *uint64 `protobuf:"varint,2,req,name=request_id" json:"request_id,omitempty"` + RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` Authoritative *bool `protobuf:"varint,3,opt,name=authoritative,def=0" json:"authoritative,omitempty"` // TODO - Remove original_principal, original_auth_data, original_auth_method // Original principal that was verified by // a Pulsar proxy. - OriginalPrincipal *string `protobuf:"bytes,4,opt,name=original_principal" json:"original_principal,omitempty"` + OriginalPrincipal *string `protobuf:"bytes,4,opt,name=original_principal,json=originalPrincipal" json:"original_principal,omitempty"` // Original auth role and auth Method that was passed // to the proxy. - OriginalAuthData *string `protobuf:"bytes,5,opt,name=original_auth_data" json:"original_auth_data,omitempty"` - OriginalAuthMethod *string `protobuf:"bytes,6,opt,name=original_auth_method" json:"original_auth_method,omitempty"` - // net mode setting - AdvertisedListenerName *string `protobuf:"bytes,7,opt,name=advertised_listener_name" json:"advertised_listener_name,omitempty"` + OriginalAuthData *string `protobuf:"bytes,5,opt,name=original_auth_data,json=originalAuthData" json:"original_auth_data,omitempty"` + OriginalAuthMethod *string `protobuf:"bytes,6,opt,name=original_auth_method,json=originalAuthMethod" json:"original_auth_method,omitempty"` + // + AdvertisedListenerName *string `protobuf:"bytes,7,opt,name=advertised_listener_name,json=advertisedListenerName" json:"advertised_listener_name,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2567,7 +2794,7 @@ func (m *CommandLookupTopic) Reset() { *m = CommandLookupTopic{} } func (m *CommandLookupTopic) String() string { return proto.CompactTextString(m) } func (*CommandLookupTopic) ProtoMessage() {} func (*CommandLookupTopic) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{18} + return fileDescriptor_39529ba7ad9caeb8, []int{19} } func (m *CommandLookupTopic) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2651,14 +2878,14 @@ type CommandLookupTopicResponse struct { BrokerServiceUrl *string `protobuf:"bytes,1,opt,name=brokerServiceUrl" json:"brokerServiceUrl,omitempty"` BrokerServiceUrlTls *string `protobuf:"bytes,2,opt,name=brokerServiceUrlTls" json:"brokerServiceUrlTls,omitempty"` Response *CommandLookupTopicResponse_LookupType `protobuf:"varint,3,opt,name=response,enum=pulsar.proto.CommandLookupTopicResponse_LookupType" json:"response,omitempty"` - RequestId *uint64 `protobuf:"varint,4,req,name=request_id" json:"request_id,omitempty"` + RequestId *uint64 `protobuf:"varint,4,req,name=request_id,json=requestId" json:"request_id,omitempty"` Authoritative *bool `protobuf:"varint,5,opt,name=authoritative,def=0" json:"authoritative,omitempty"` Error *ServerError `protobuf:"varint,6,opt,name=error,enum=pulsar.proto.ServerError" json:"error,omitempty"` Message *string `protobuf:"bytes,7,opt,name=message" json:"message,omitempty"` // If it's true, indicates to the client that it must // always connect through the service url after the // lookup has been completed. - ProxyThroughServiceUrl *bool `protobuf:"varint,8,opt,name=proxy_through_service_url,def=0" json:"proxy_through_service_url,omitempty"` + ProxyThroughServiceUrl *bool `protobuf:"varint,8,opt,name=proxy_through_service_url,json=proxyThroughServiceUrl,def=0" json:"proxy_through_service_url,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2668,7 +2895,7 @@ func (m *CommandLookupTopicResponse) Reset() { *m = CommandLookupTopicRe func (m *CommandLookupTopicResponse) String() string { return proto.CompactTextString(m) } func (*CommandLookupTopicResponse) ProtoMessage() {} func (*CommandLookupTopicResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{19} + return fileDescriptor_39529ba7ad9caeb8, []int{20} } func (m *CommandLookupTopicResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2760,11 +2987,11 @@ func (m *CommandLookupTopicResponse) GetProxyThroughServiceUrl() bool { /// all messages sent with this producer_id will be persisted on the topic type CommandProducer struct { Topic *string `protobuf:"bytes,1,req,name=topic" json:"topic,omitempty"` - ProducerId *uint64 `protobuf:"varint,2,req,name=producer_id" json:"producer_id,omitempty"` - RequestId *uint64 `protobuf:"varint,3,req,name=request_id" json:"request_id,omitempty"` + ProducerId *uint64 `protobuf:"varint,2,req,name=producer_id,json=producerId" json:"producer_id,omitempty"` + RequestId *uint64 `protobuf:"varint,3,req,name=request_id,json=requestId" json:"request_id,omitempty"` /// If a producer name is specified, the name will be used, /// otherwise the broker will generate a unique name - ProducerName *string `protobuf:"bytes,4,opt,name=producer_name" json:"producer_name,omitempty"` + ProducerName *string `protobuf:"bytes,4,opt,name=producer_name,json=producerName" json:"producer_name,omitempty"` Encrypted *bool `protobuf:"varint,5,opt,name=encrypted,def=0" json:"encrypted,omitempty"` /// Add optional metadata key=value to this producer Metadata []*KeyValue `protobuf:"bytes,6,rep,name=metadata" json:"metadata,omitempty"` @@ -2773,17 +3000,25 @@ type CommandProducer struct { Epoch *uint64 `protobuf:"varint,8,opt,name=epoch,def=0" json:"epoch,omitempty"` // Indicate the name of the producer is generated or user provided // Use default true here is in order to be forward compatible with the client - UserProvidedProducerName *bool `protobuf:"varint,9,opt,name=user_provided_producer_name,def=1" json:"user_provided_producer_name,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + UserProvidedProducerName *bool `protobuf:"varint,9,opt,name=user_provided_producer_name,json=userProvidedProducerName,def=1" json:"user_provided_producer_name,omitempty"` + // Require that this producers will be the only producer allowed on the topic + ProducerAccessMode *ProducerAccessMode `protobuf:"varint,10,opt,name=producer_access_mode,json=producerAccessMode,enum=pulsar.proto.ProducerAccessMode,def=0" json:"producer_access_mode,omitempty"` + // Topic epoch is used to fence off producers that reconnects after a new + // exclusive producer has already taken over. This id is assigned by the + // broker on the CommandProducerSuccess. The first time, the client will + // leave it empty and then it will always carry the same epoch number on + // the subsequent reconnections. + TopicEpoch *uint64 `protobuf:"varint,11,opt,name=topic_epoch,json=topicEpoch" json:"topic_epoch,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *CommandProducer) Reset() { *m = CommandProducer{} } func (m *CommandProducer) String() string { return proto.CompactTextString(m) } func (*CommandProducer) ProtoMessage() {} func (*CommandProducer) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{20} + return fileDescriptor_39529ba7ad9caeb8, []int{21} } func (m *CommandProducer) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2815,6 +3050,7 @@ var xxx_messageInfo_CommandProducer proto.InternalMessageInfo const Default_CommandProducer_Encrypted bool = false const Default_CommandProducer_Epoch uint64 = 0 const Default_CommandProducer_UserProvidedProducerName bool = true +const Default_CommandProducer_ProducerAccessMode ProducerAccessMode = ProducerAccessMode_Shared func (m *CommandProducer) GetTopic() string { if m != nil && m.Topic != nil { @@ -2879,14 +3115,29 @@ func (m *CommandProducer) GetUserProvidedProducerName() bool { return Default_CommandProducer_UserProvidedProducerName } +func (m *CommandProducer) GetProducerAccessMode() ProducerAccessMode { + if m != nil && m.ProducerAccessMode != nil { + return *m.ProducerAccessMode + } + return Default_CommandProducer_ProducerAccessMode +} + +func (m *CommandProducer) GetTopicEpoch() uint64 { + if m != nil && m.TopicEpoch != nil { + return *m.TopicEpoch + } + return 0 +} + type CommandSend struct { - ProducerId *uint64 `protobuf:"varint,1,req,name=producer_id" json:"producer_id,omitempty"` - SequenceId *uint64 `protobuf:"varint,2,req,name=sequence_id" json:"sequence_id,omitempty"` - NumMessages *int32 `protobuf:"varint,3,opt,name=num_messages,def=1" json:"num_messages,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,4,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,5,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + ProducerId *uint64 `protobuf:"varint,1,req,name=producer_id,json=producerId" json:"producer_id,omitempty"` + SequenceId *uint64 `protobuf:"varint,2,req,name=sequence_id,json=sequenceId" json:"sequence_id,omitempty"` + NumMessages *int32 `protobuf:"varint,3,opt,name=num_messages,json=numMessages,def=1" json:"num_messages,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,4,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,5,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` /// Add highest sequence id to support batch message with external sequence id - HighestSequenceId *uint64 `protobuf:"varint,6,opt,name=highest_sequence_id,def=0" json:"highest_sequence_id,omitempty"` + HighestSequenceId *uint64 `protobuf:"varint,6,opt,name=highest_sequence_id,json=highestSequenceId,def=0" json:"highest_sequence_id,omitempty"` + IsChunk *bool `protobuf:"varint,7,opt,name=is_chunk,json=isChunk,def=0" json:"is_chunk,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2896,7 +3147,7 @@ func (m *CommandSend) Reset() { *m = CommandSend{} } func (m *CommandSend) String() string { return proto.CompactTextString(m) } func (*CommandSend) ProtoMessage() {} func (*CommandSend) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{21} + return fileDescriptor_39529ba7ad9caeb8, []int{22} } func (m *CommandSend) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2929,6 +3180,7 @@ const Default_CommandSend_NumMessages int32 = 1 const Default_CommandSend_TxnidLeastBits uint64 = 0 const Default_CommandSend_TxnidMostBits uint64 = 0 const Default_CommandSend_HighestSequenceId uint64 = 0 +const Default_CommandSend_IsChunk bool = false func (m *CommandSend) GetProducerId() uint64 { if m != nil && m.ProducerId != nil { @@ -2972,11 +3224,18 @@ func (m *CommandSend) GetHighestSequenceId() uint64 { return Default_CommandSend_HighestSequenceId } +func (m *CommandSend) GetIsChunk() bool { + if m != nil && m.IsChunk != nil { + return *m.IsChunk + } + return Default_CommandSend_IsChunk +} + type CommandSendReceipt struct { - ProducerId *uint64 `protobuf:"varint,1,req,name=producer_id" json:"producer_id,omitempty"` - SequenceId *uint64 `protobuf:"varint,2,req,name=sequence_id" json:"sequence_id,omitempty"` - MessageId *MessageIdData `protobuf:"bytes,3,opt,name=message_id" json:"message_id,omitempty"` - HighestSequenceId *uint64 `protobuf:"varint,4,opt,name=highest_sequence_id,def=0" json:"highest_sequence_id,omitempty"` + ProducerId *uint64 `protobuf:"varint,1,req,name=producer_id,json=producerId" json:"producer_id,omitempty"` + SequenceId *uint64 `protobuf:"varint,2,req,name=sequence_id,json=sequenceId" json:"sequence_id,omitempty"` + MessageId *MessageIdData `protobuf:"bytes,3,opt,name=message_id,json=messageId" json:"message_id,omitempty"` + HighestSequenceId *uint64 `protobuf:"varint,4,opt,name=highest_sequence_id,json=highestSequenceId,def=0" json:"highest_sequence_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2986,7 +3245,7 @@ func (m *CommandSendReceipt) Reset() { *m = CommandSendReceipt{} } func (m *CommandSendReceipt) String() string { return proto.CompactTextString(m) } func (*CommandSendReceipt) ProtoMessage() {} func (*CommandSendReceipt) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{22} + return fileDescriptor_39529ba7ad9caeb8, []int{23} } func (m *CommandSendReceipt) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3046,8 +3305,8 @@ func (m *CommandSendReceipt) GetHighestSequenceId() uint64 { } type CommandSendError struct { - ProducerId *uint64 `protobuf:"varint,1,req,name=producer_id" json:"producer_id,omitempty"` - SequenceId *uint64 `protobuf:"varint,2,req,name=sequence_id" json:"sequence_id,omitempty"` + ProducerId *uint64 `protobuf:"varint,1,req,name=producer_id,json=producerId" json:"producer_id,omitempty"` + SequenceId *uint64 `protobuf:"varint,2,req,name=sequence_id,json=sequenceId" json:"sequence_id,omitempty"` Error *ServerError `protobuf:"varint,3,req,name=error,enum=pulsar.proto.ServerError" json:"error,omitempty"` Message *string `protobuf:"bytes,4,req,name=message" json:"message,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -3059,7 +3318,7 @@ func (m *CommandSendError) Reset() { *m = CommandSendError{} } func (m *CommandSendError) String() string { return proto.CompactTextString(m) } func (*CommandSendError) ProtoMessage() {} func (*CommandSendError) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{23} + return fileDescriptor_39529ba7ad9caeb8, []int{24} } func (m *CommandSendError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3117,9 +3376,10 @@ func (m *CommandSendError) GetMessage() string { } type CommandMessage struct { - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id" json:"consumer_id,omitempty"` - MessageId *MessageIdData `protobuf:"bytes,2,req,name=message_id" json:"message_id,omitempty"` - RedeliveryCount *uint32 `protobuf:"varint,3,opt,name=redelivery_count,def=0" json:"redelivery_count,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` + MessageId *MessageIdData `protobuf:"bytes,2,req,name=message_id,json=messageId" json:"message_id,omitempty"` + RedeliveryCount *uint32 `protobuf:"varint,3,opt,name=redelivery_count,json=redeliveryCount,def=0" json:"redelivery_count,omitempty"` + AckSet []int64 `protobuf:"varint,4,rep,name=ack_set,json=ackSet" json:"ack_set,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -3129,7 +3389,7 @@ func (m *CommandMessage) Reset() { *m = CommandMessage{} } func (m *CommandMessage) String() string { return proto.CompactTextString(m) } func (*CommandMessage) ProtoMessage() {} func (*CommandMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{24} + return fileDescriptor_39529ba7ad9caeb8, []int{25} } func (m *CommandMessage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3181,15 +3441,23 @@ func (m *CommandMessage) GetRedeliveryCount() uint32 { return Default_CommandMessage_RedeliveryCount } +func (m *CommandMessage) GetAckSet() []int64 { + if m != nil { + return m.AckSet + } + return nil +} + type CommandAck struct { - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id" json:"consumer_id,omitempty"` - AckType *CommandAck_AckType `protobuf:"varint,2,req,name=ack_type,enum=pulsar.proto.CommandAck_AckType" json:"ack_type,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` + AckType *CommandAck_AckType `protobuf:"varint,2,req,name=ack_type,json=ackType,enum=pulsar.proto.CommandAck_AckType" json:"ack_type,omitempty"` // In case of individual acks, the client can pass a list of message ids - MessageId []*MessageIdData `protobuf:"bytes,3,rep,name=message_id" json:"message_id,omitempty"` - ValidationError *CommandAck_ValidationError `protobuf:"varint,4,opt,name=validation_error,enum=pulsar.proto.CommandAck_ValidationError" json:"validation_error,omitempty"` + MessageId []*MessageIdData `protobuf:"bytes,3,rep,name=message_id,json=messageId" json:"message_id,omitempty"` + ValidationError *CommandAck_ValidationError `protobuf:"varint,4,opt,name=validation_error,json=validationError,enum=pulsar.proto.CommandAck_ValidationError" json:"validation_error,omitempty"` Properties []*KeyLongValue `protobuf:"bytes,5,rep,name=properties" json:"properties,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,6,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,7,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,6,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,7,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` + RequestId *uint64 `protobuf:"varint,8,opt,name=request_id,json=requestId" json:"request_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -3199,7 +3467,7 @@ func (m *CommandAck) Reset() { *m = CommandAck{} } func (m *CommandAck) String() string { return proto.CompactTextString(m) } func (*CommandAck) ProtoMessage() {} func (*CommandAck) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{25} + return fileDescriptor_39529ba7ad9caeb8, []int{26} } func (m *CommandAck) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3280,12 +3548,20 @@ func (m *CommandAck) GetTxnidMostBits() uint64 { return Default_CommandAck_TxnidMostBits } +func (m *CommandAck) GetRequestId() uint64 { + if m != nil && m.RequestId != nil { + return *m.RequestId + } + return 0 +} + type CommandAckResponse struct { - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id" json:"consumer_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` Error *ServerError `protobuf:"varint,4,opt,name=error,enum=pulsar.proto.ServerError" json:"error,omitempty"` Message *string `protobuf:"bytes,5,opt,name=message" json:"message,omitempty"` + RequestId *uint64 `protobuf:"varint,6,opt,name=request_id,json=requestId" json:"request_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -3295,7 +3571,7 @@ func (m *CommandAckResponse) Reset() { *m = CommandAckResponse{} } func (m *CommandAckResponse) String() string { return proto.CompactTextString(m) } func (*CommandAckResponse) ProtoMessage() {} func (*CommandAckResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{26} + return fileDescriptor_39529ba7ad9caeb8, []int{27} } func (m *CommandAckResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3362,10 +3638,17 @@ func (m *CommandAckResponse) GetMessage() string { return "" } +func (m *CommandAckResponse) GetRequestId() uint64 { + if m != nil && m.RequestId != nil { + return *m.RequestId + } + return 0 +} + // changes on active consumer type CommandActiveConsumerChange struct { - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id" json:"consumer_id,omitempty"` - IsActive *bool `protobuf:"varint,2,opt,name=is_active,def=0" json:"is_active,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` + IsActive *bool `protobuf:"varint,2,opt,name=is_active,json=isActive,def=0" json:"is_active,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -3375,7 +3658,7 @@ func (m *CommandActiveConsumerChange) Reset() { *m = CommandActiveConsum func (m *CommandActiveConsumerChange) String() string { return proto.CompactTextString(m) } func (*CommandActiveConsumerChange) ProtoMessage() {} func (*CommandActiveConsumerChange) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{27} + return fileDescriptor_39529ba7ad9caeb8, []int{28} } func (m *CommandActiveConsumerChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3421,7 +3704,7 @@ func (m *CommandActiveConsumerChange) GetIsActive() bool { } type CommandFlow struct { - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id" json:"consumer_id,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` // Max number of messages to prefetch, in addition // of any number previously specified MessagePermits *uint32 `protobuf:"varint,2,req,name=messagePermits" json:"messagePermits,omitempty"` @@ -3434,7 +3717,7 @@ func (m *CommandFlow) Reset() { *m = CommandFlow{} } func (m *CommandFlow) String() string { return proto.CompactTextString(m) } func (*CommandFlow) ProtoMessage() {} func (*CommandFlow) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{28} + return fileDescriptor_39529ba7ad9caeb8, []int{29} } func (m *CommandFlow) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3478,8 +3761,8 @@ func (m *CommandFlow) GetMessagePermits() uint32 { } type CommandUnsubscribe struct { - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id" json:"consumer_id,omitempty"` - RequestId *uint64 `protobuf:"varint,2,req,name=request_id" json:"request_id,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` + RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -3489,7 +3772,7 @@ func (m *CommandUnsubscribe) Reset() { *m = CommandUnsubscribe{} } func (m *CommandUnsubscribe) String() string { return proto.CompactTextString(m) } func (*CommandUnsubscribe) ProtoMessage() {} func (*CommandUnsubscribe) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{29} + return fileDescriptor_39529ba7ad9caeb8, []int{30} } func (m *CommandUnsubscribe) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3534,10 +3817,10 @@ func (m *CommandUnsubscribe) GetRequestId() uint64 { // Reset an existing consumer to a particular message id type CommandSeek struct { - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id" json:"consumer_id,omitempty"` - RequestId *uint64 `protobuf:"varint,2,req,name=request_id" json:"request_id,omitempty"` - MessageId *MessageIdData `protobuf:"bytes,3,opt,name=message_id" json:"message_id,omitempty"` - MessagePublishTime *uint64 `protobuf:"varint,4,opt,name=message_publish_time" json:"message_publish_time,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` + RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` + MessageId *MessageIdData `protobuf:"bytes,3,opt,name=message_id,json=messageId" json:"message_id,omitempty"` + MessagePublishTime *uint64 `protobuf:"varint,4,opt,name=message_publish_time,json=messagePublishTime" json:"message_publish_time,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -3547,7 +3830,7 @@ func (m *CommandSeek) Reset() { *m = CommandSeek{} } func (m *CommandSeek) String() string { return proto.CompactTextString(m) } func (*CommandSeek) ProtoMessage() {} func (*CommandSeek) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{30} + return fileDescriptor_39529ba7ad9caeb8, []int{31} } func (m *CommandSeek) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3608,7 +3891,7 @@ func (m *CommandSeek) GetMessagePublishTime() uint64 { // has been forcefully terminated and there are no more // messages left to consume type CommandReachedEndOfTopic struct { - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id" json:"consumer_id,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -3618,7 +3901,7 @@ func (m *CommandReachedEndOfTopic) Reset() { *m = CommandReachedEndOfTop func (m *CommandReachedEndOfTopic) String() string { return proto.CompactTextString(m) } func (*CommandReachedEndOfTopic) ProtoMessage() {} func (*CommandReachedEndOfTopic) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{31} + return fileDescriptor_39529ba7ad9caeb8, []int{32} } func (m *CommandReachedEndOfTopic) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3655,8 +3938,8 @@ func (m *CommandReachedEndOfTopic) GetConsumerId() uint64 { } type CommandCloseProducer struct { - ProducerId *uint64 `protobuf:"varint,1,req,name=producer_id" json:"producer_id,omitempty"` - RequestId *uint64 `protobuf:"varint,2,req,name=request_id" json:"request_id,omitempty"` + ProducerId *uint64 `protobuf:"varint,1,req,name=producer_id,json=producerId" json:"producer_id,omitempty"` + RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -3666,7 +3949,7 @@ func (m *CommandCloseProducer) Reset() { *m = CommandCloseProducer{} } func (m *CommandCloseProducer) String() string { return proto.CompactTextString(m) } func (*CommandCloseProducer) ProtoMessage() {} func (*CommandCloseProducer) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{32} + return fileDescriptor_39529ba7ad9caeb8, []int{33} } func (m *CommandCloseProducer) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3710,8 +3993,8 @@ func (m *CommandCloseProducer) GetRequestId() uint64 { } type CommandCloseConsumer struct { - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id" json:"consumer_id,omitempty"` - RequestId *uint64 `protobuf:"varint,2,req,name=request_id" json:"request_id,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` + RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -3721,7 +4004,7 @@ func (m *CommandCloseConsumer) Reset() { *m = CommandCloseConsumer{} } func (m *CommandCloseConsumer) String() string { return proto.CompactTextString(m) } func (*CommandCloseConsumer) ProtoMessage() {} func (*CommandCloseConsumer) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{33} + return fileDescriptor_39529ba7ad9caeb8, []int{34} } func (m *CommandCloseConsumer) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3765,8 +4048,8 @@ func (m *CommandCloseConsumer) GetRequestId() uint64 { } type CommandRedeliverUnacknowledgedMessages struct { - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id" json:"consumer_id,omitempty"` - MessageIds []*MessageIdData `protobuf:"bytes,2,rep,name=message_ids" json:"message_ids,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` + MessageIds []*MessageIdData `protobuf:"bytes,2,rep,name=message_ids,json=messageIds" json:"message_ids,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -3778,7 +4061,7 @@ func (m *CommandRedeliverUnacknowledgedMessages) Reset() { func (m *CommandRedeliverUnacknowledgedMessages) String() string { return proto.CompactTextString(m) } func (*CommandRedeliverUnacknowledgedMessages) ProtoMessage() {} func (*CommandRedeliverUnacknowledgedMessages) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{34} + return fileDescriptor_39529ba7ad9caeb8, []int{35} } func (m *CommandRedeliverUnacknowledgedMessages) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3822,7 +4105,7 @@ func (m *CommandRedeliverUnacknowledgedMessages) GetMessageIds() []*MessageIdDat } type CommandSuccess struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` Schema *Schema `protobuf:"bytes,2,opt,name=schema" json:"schema,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -3833,7 +4116,7 @@ func (m *CommandSuccess) Reset() { *m = CommandSuccess{} } func (m *CommandSuccess) String() string { return proto.CompactTextString(m) } func (*CommandSuccess) ProtoMessage() {} func (*CommandSuccess) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{35} + return fileDescriptor_39529ba7ad9caeb8, []int{36} } func (m *CommandSuccess) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3878,12 +4161,19 @@ func (m *CommandSuccess) GetSchema() *Schema { /// Response from CommandProducer type CommandProducerSuccess struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - ProducerName *string `protobuf:"bytes,2,req,name=producer_name" json:"producer_name,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + ProducerName *string `protobuf:"bytes,2,req,name=producer_name,json=producerName" json:"producer_name,omitempty"` // The last sequence id that was stored by this producer in the previous session // This will only be meaningful if deduplication has been enabled. - LastSequenceId *int64 `protobuf:"varint,3,opt,name=last_sequence_id,def=-1" json:"last_sequence_id,omitempty"` - SchemaVersion []byte `protobuf:"bytes,4,opt,name=schema_version" json:"schema_version,omitempty"` + LastSequenceId *int64 `protobuf:"varint,3,opt,name=last_sequence_id,json=lastSequenceId,def=-1" json:"last_sequence_id,omitempty"` + SchemaVersion []byte `protobuf:"bytes,4,opt,name=schema_version,json=schemaVersion" json:"schema_version,omitempty"` + // The topic epoch assigned by the broker. This field will only be set if we + // were requiring exclusive access when creating the producer. + TopicEpoch *uint64 `protobuf:"varint,5,opt,name=topic_epoch,json=topicEpoch" json:"topic_epoch,omitempty"` + // If producer is not "ready", the client will avoid to timeout the request + // for creating the producer. Instead it will wait indefinitely until it gets + // a subsequent `CommandProducerSuccess` with `producer_ready==true`. + ProducerReady *bool `protobuf:"varint,6,opt,name=producer_ready,json=producerReady,def=1" json:"producer_ready,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -3893,7 +4183,7 @@ func (m *CommandProducerSuccess) Reset() { *m = CommandProducerSuccess{} func (m *CommandProducerSuccess) String() string { return proto.CompactTextString(m) } func (*CommandProducerSuccess) ProtoMessage() {} func (*CommandProducerSuccess) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{36} + return fileDescriptor_39529ba7ad9caeb8, []int{37} } func (m *CommandProducerSuccess) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3923,6 +4213,7 @@ func (m *CommandProducerSuccess) XXX_DiscardUnknown() { var xxx_messageInfo_CommandProducerSuccess proto.InternalMessageInfo const Default_CommandProducerSuccess_LastSequenceId int64 = -1 +const Default_CommandProducerSuccess_ProducerReady bool = true func (m *CommandProducerSuccess) GetRequestId() uint64 { if m != nil && m.RequestId != nil { @@ -3952,8 +4243,22 @@ func (m *CommandProducerSuccess) GetSchemaVersion() []byte { return nil } +func (m *CommandProducerSuccess) GetTopicEpoch() uint64 { + if m != nil && m.TopicEpoch != nil { + return *m.TopicEpoch + } + return 0 +} + +func (m *CommandProducerSuccess) GetProducerReady() bool { + if m != nil && m.ProducerReady != nil { + return *m.ProducerReady + } + return Default_CommandProducerSuccess_ProducerReady +} + type CommandError struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` Error *ServerError `protobuf:"varint,2,req,name=error,enum=pulsar.proto.ServerError" json:"error,omitempty"` Message *string `protobuf:"bytes,3,req,name=message" json:"message,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -3965,7 +4270,7 @@ func (m *CommandError) Reset() { *m = CommandError{} } func (m *CommandError) String() string { return proto.CompactTextString(m) } func (*CommandError) ProtoMessage() {} func (*CommandError) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{37} + return fileDescriptor_39529ba7ad9caeb8, []int{38} } func (m *CommandError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4028,7 +4333,7 @@ func (m *CommandPing) Reset() { *m = CommandPing{} } func (m *CommandPing) String() string { return proto.CompactTextString(m) } func (*CommandPing) ProtoMessage() {} func (*CommandPing) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{38} + return fileDescriptor_39529ba7ad9caeb8, []int{39} } func (m *CommandPing) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4067,7 +4372,7 @@ func (m *CommandPong) Reset() { *m = CommandPong{} } func (m *CommandPong) String() string { return proto.CompactTextString(m) } func (*CommandPong) ProtoMessage() {} func (*CommandPong) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{39} + return fileDescriptor_39529ba7ad9caeb8, []int{40} } func (m *CommandPong) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4097,10 +4402,10 @@ func (m *CommandPong) XXX_DiscardUnknown() { var xxx_messageInfo_CommandPong proto.InternalMessageInfo type CommandConsumerStats struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` // required string topic_name = 2; // required string subscription_name = 3; - ConsumerId *uint64 `protobuf:"varint,4,req,name=consumer_id" json:"consumer_id,omitempty"` + ConsumerId *uint64 `protobuf:"varint,4,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -4110,7 +4415,7 @@ func (m *CommandConsumerStats) Reset() { *m = CommandConsumerStats{} } func (m *CommandConsumerStats) String() string { return proto.CompactTextString(m) } func (*CommandConsumerStats) ProtoMessage() {} func (*CommandConsumerStats) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{40} + return fileDescriptor_39529ba7ad9caeb8, []int{41} } func (m *CommandConsumerStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4154,9 +4459,9 @@ func (m *CommandConsumerStats) GetConsumerId() uint64 { } type CommandConsumerStatsResponse struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - ErrorCode *ServerError `protobuf:"varint,2,opt,name=error_code,enum=pulsar.proto.ServerError" json:"error_code,omitempty"` - ErrorMessage *string `protobuf:"bytes,3,opt,name=error_message" json:"error_message,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + ErrorCode *ServerError `protobuf:"varint,2,opt,name=error_code,json=errorCode,enum=pulsar.proto.ServerError" json:"error_code,omitempty"` + ErrorMessage *string `protobuf:"bytes,3,opt,name=error_message,json=errorMessage" json:"error_message,omitempty"` /// Total rate of messages delivered to the consumer. msg/s MsgRateOut *float64 `protobuf:"fixed64,4,opt,name=msgRateOut" json:"msgRateOut,omitempty"` /// Total throughput delivered to the consumer. bytes/s @@ -4190,7 +4495,7 @@ func (m *CommandConsumerStatsResponse) Reset() { *m = CommandConsumerSta func (m *CommandConsumerStatsResponse) String() string { return proto.CompactTextString(m) } func (*CommandConsumerStatsResponse) ProtoMessage() {} func (*CommandConsumerStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{41} + return fileDescriptor_39529ba7ad9caeb8, []int{42} } func (m *CommandConsumerStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4325,8 +4630,8 @@ func (m *CommandConsumerStatsResponse) GetMsgBacklog() uint64 { } type CommandGetLastMessageId struct { - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id" json:"consumer_id,omitempty"` - RequestId *uint64 `protobuf:"varint,2,req,name=request_id" json:"request_id,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` + RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -4336,7 +4641,7 @@ func (m *CommandGetLastMessageId) Reset() { *m = CommandGetLastMessageId func (m *CommandGetLastMessageId) String() string { return proto.CompactTextString(m) } func (*CommandGetLastMessageId) ProtoMessage() {} func (*CommandGetLastMessageId) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{42} + return fileDescriptor_39529ba7ad9caeb8, []int{43} } func (m *CommandGetLastMessageId) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4380,18 +4685,19 @@ func (m *CommandGetLastMessageId) GetRequestId() uint64 { } type CommandGetLastMessageIdResponse struct { - LastMessageId *MessageIdData `protobuf:"bytes,1,req,name=last_message_id" json:"last_message_id,omitempty"` - RequestId *uint64 `protobuf:"varint,2,req,name=request_id" json:"request_id,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + LastMessageId *MessageIdData `protobuf:"bytes,1,req,name=last_message_id,json=lastMessageId" json:"last_message_id,omitempty"` + RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` + ConsumerMarkDeletePosition *MessageIdData `protobuf:"bytes,3,opt,name=consumer_mark_delete_position,json=consumerMarkDeletePosition" json:"consumer_mark_delete_position,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *CommandGetLastMessageIdResponse) Reset() { *m = CommandGetLastMessageIdResponse{} } func (m *CommandGetLastMessageIdResponse) String() string { return proto.CompactTextString(m) } func (*CommandGetLastMessageIdResponse) ProtoMessage() {} func (*CommandGetLastMessageIdResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{43} + return fileDescriptor_39529ba7ad9caeb8, []int{44} } func (m *CommandGetLastMessageIdResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4434,8 +4740,15 @@ func (m *CommandGetLastMessageIdResponse) GetRequestId() uint64 { return 0 } +func (m *CommandGetLastMessageIdResponse) GetConsumerMarkDeletePosition() *MessageIdData { + if m != nil { + return m.ConsumerMarkDeletePosition + } + return nil +} + type CommandGetTopicsOfNamespace struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` Namespace *string `protobuf:"bytes,2,req,name=namespace" json:"namespace,omitempty"` Mode *CommandGetTopicsOfNamespace_Mode `protobuf:"varint,3,opt,name=mode,enum=pulsar.proto.CommandGetTopicsOfNamespace_Mode,def=0" json:"mode,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -4447,7 +4760,7 @@ func (m *CommandGetTopicsOfNamespace) Reset() { *m = CommandGetTopicsOfN func (m *CommandGetTopicsOfNamespace) String() string { return proto.CompactTextString(m) } func (*CommandGetTopicsOfNamespace) ProtoMessage() {} func (*CommandGetTopicsOfNamespace) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{44} + return fileDescriptor_39529ba7ad9caeb8, []int{45} } func (m *CommandGetTopicsOfNamespace) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4500,7 +4813,7 @@ func (m *CommandGetTopicsOfNamespace) GetMode() CommandGetTopicsOfNamespace_Mode } type CommandGetTopicsOfNamespaceResponse struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` Topics []string `protobuf:"bytes,2,rep,name=topics" json:"topics,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -4511,7 +4824,7 @@ func (m *CommandGetTopicsOfNamespaceResponse) Reset() { *m = CommandGetT func (m *CommandGetTopicsOfNamespaceResponse) String() string { return proto.CompactTextString(m) } func (*CommandGetTopicsOfNamespaceResponse) ProtoMessage() {} func (*CommandGetTopicsOfNamespaceResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{45} + return fileDescriptor_39529ba7ad9caeb8, []int{46} } func (m *CommandGetTopicsOfNamespaceResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4555,9 +4868,9 @@ func (m *CommandGetTopicsOfNamespaceResponse) GetTopics() []string { } type CommandGetSchema struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` Topic *string `protobuf:"bytes,2,req,name=topic" json:"topic,omitempty"` - SchemaVersion []byte `protobuf:"bytes,3,opt,name=schema_version" json:"schema_version,omitempty"` + SchemaVersion []byte `protobuf:"bytes,3,opt,name=schema_version,json=schemaVersion" json:"schema_version,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -4567,7 +4880,7 @@ func (m *CommandGetSchema) Reset() { *m = CommandGetSchema{} } func (m *CommandGetSchema) String() string { return proto.CompactTextString(m) } func (*CommandGetSchema) ProtoMessage() {} func (*CommandGetSchema) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{46} + return fileDescriptor_39529ba7ad9caeb8, []int{47} } func (m *CommandGetSchema) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4618,11 +4931,11 @@ func (m *CommandGetSchema) GetSchemaVersion() []byte { } type CommandGetSchemaResponse struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - ErrorCode *ServerError `protobuf:"varint,2,opt,name=error_code,enum=pulsar.proto.ServerError" json:"error_code,omitempty"` - ErrorMessage *string `protobuf:"bytes,3,opt,name=error_message" json:"error_message,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + ErrorCode *ServerError `protobuf:"varint,2,opt,name=error_code,json=errorCode,enum=pulsar.proto.ServerError" json:"error_code,omitempty"` + ErrorMessage *string `protobuf:"bytes,3,opt,name=error_message,json=errorMessage" json:"error_message,omitempty"` Schema *Schema `protobuf:"bytes,4,opt,name=schema" json:"schema,omitempty"` - SchemaVersion []byte `protobuf:"bytes,5,opt,name=schema_version" json:"schema_version,omitempty"` + SchemaVersion []byte `protobuf:"bytes,5,opt,name=schema_version,json=schemaVersion" json:"schema_version,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -4632,7 +4945,7 @@ func (m *CommandGetSchemaResponse) Reset() { *m = CommandGetSchemaRespon func (m *CommandGetSchemaResponse) String() string { return proto.CompactTextString(m) } func (*CommandGetSchemaResponse) ProtoMessage() {} func (*CommandGetSchemaResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{47} + return fileDescriptor_39529ba7ad9caeb8, []int{48} } func (m *CommandGetSchemaResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4697,7 +5010,7 @@ func (m *CommandGetSchemaResponse) GetSchemaVersion() []byte { } type CommandGetOrCreateSchema struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` Topic *string `protobuf:"bytes,2,req,name=topic" json:"topic,omitempty"` Schema *Schema `protobuf:"bytes,3,req,name=schema" json:"schema,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -4709,7 +5022,7 @@ func (m *CommandGetOrCreateSchema) Reset() { *m = CommandGetOrCreateSche func (m *CommandGetOrCreateSchema) String() string { return proto.CompactTextString(m) } func (*CommandGetOrCreateSchema) ProtoMessage() {} func (*CommandGetOrCreateSchema) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{48} + return fileDescriptor_39529ba7ad9caeb8, []int{49} } func (m *CommandGetOrCreateSchema) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4760,10 +5073,10 @@ func (m *CommandGetOrCreateSchema) GetSchema() *Schema { } type CommandGetOrCreateSchemaResponse struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - ErrorCode *ServerError `protobuf:"varint,2,opt,name=error_code,enum=pulsar.proto.ServerError" json:"error_code,omitempty"` - ErrorMessage *string `protobuf:"bytes,3,opt,name=error_message" json:"error_message,omitempty"` - SchemaVersion []byte `protobuf:"bytes,4,opt,name=schema_version" json:"schema_version,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + ErrorCode *ServerError `protobuf:"varint,2,opt,name=error_code,json=errorCode,enum=pulsar.proto.ServerError" json:"error_code,omitempty"` + ErrorMessage *string `protobuf:"bytes,3,opt,name=error_message,json=errorMessage" json:"error_message,omitempty"` + SchemaVersion []byte `protobuf:"bytes,4,opt,name=schema_version,json=schemaVersion" json:"schema_version,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -4773,7 +5086,7 @@ func (m *CommandGetOrCreateSchemaResponse) Reset() { *m = CommandGetOrCr func (m *CommandGetOrCreateSchemaResponse) String() string { return proto.CompactTextString(m) } func (*CommandGetOrCreateSchemaResponse) ProtoMessage() {} func (*CommandGetOrCreateSchemaResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{49} + return fileDescriptor_39529ba7ad9caeb8, []int{50} } func (m *CommandGetOrCreateSchemaResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4831,9 +5144,9 @@ func (m *CommandGetOrCreateSchemaResponse) GetSchemaVersion() []byte { } type CommandNewTxn struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnTtlSeconds *uint64 `protobuf:"varint,2,opt,name=txn_ttl_seconds,def=0" json:"txn_ttl_seconds,omitempty"` - TcId *uint64 `protobuf:"varint,3,opt,name=tc_id,def=0" json:"tc_id,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnTtlSeconds *uint64 `protobuf:"varint,2,opt,name=txn_ttl_seconds,json=txnTtlSeconds,def=0" json:"txn_ttl_seconds,omitempty"` + TcId *uint64 `protobuf:"varint,3,opt,name=tc_id,json=tcId,def=0" json:"tc_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -4843,7 +5156,7 @@ func (m *CommandNewTxn) Reset() { *m = CommandNewTxn{} } func (m *CommandNewTxn) String() string { return proto.CompactTextString(m) } func (*CommandNewTxn) ProtoMessage() {} func (*CommandNewTxn) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{50} + return fileDescriptor_39529ba7ad9caeb8, []int{51} } func (m *CommandNewTxn) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4897,9 +5210,9 @@ func (m *CommandNewTxn) GetTcId() uint64 { } type CommandNewTxnResponse struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` Error *ServerError `protobuf:"varint,4,opt,name=error,enum=pulsar.proto.ServerError" json:"error,omitempty"` Message *string `protobuf:"bytes,5,opt,name=message" json:"message,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -4911,7 +5224,7 @@ func (m *CommandNewTxnResponse) Reset() { *m = CommandNewTxnResponse{} } func (m *CommandNewTxnResponse) String() string { return proto.CompactTextString(m) } func (*CommandNewTxnResponse) ProtoMessage() {} func (*CommandNewTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{51} + return fileDescriptor_39529ba7ad9caeb8, []int{52} } func (m *CommandNewTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4979,9 +5292,9 @@ func (m *CommandNewTxnResponse) GetMessage() string { } type CommandAddPartitionToTxn struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` Partitions []string `protobuf:"bytes,4,rep,name=partitions" json:"partitions,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -4992,7 +5305,7 @@ func (m *CommandAddPartitionToTxn) Reset() { *m = CommandAddPartitionToT func (m *CommandAddPartitionToTxn) String() string { return proto.CompactTextString(m) } func (*CommandAddPartitionToTxn) ProtoMessage() {} func (*CommandAddPartitionToTxn) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{52} + return fileDescriptor_39529ba7ad9caeb8, []int{53} } func (m *CommandAddPartitionToTxn) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5053,9 +5366,9 @@ func (m *CommandAddPartitionToTxn) GetPartitions() []string { } type CommandAddPartitionToTxnResponse struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` Error *ServerError `protobuf:"varint,4,opt,name=error,enum=pulsar.proto.ServerError" json:"error,omitempty"` Message *string `protobuf:"bytes,5,opt,name=message" json:"message,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -5067,7 +5380,7 @@ func (m *CommandAddPartitionToTxnResponse) Reset() { *m = CommandAddPart func (m *CommandAddPartitionToTxnResponse) String() string { return proto.CompactTextString(m) } func (*CommandAddPartitionToTxnResponse) ProtoMessage() {} func (*CommandAddPartitionToTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{53} + return fileDescriptor_39529ba7ad9caeb8, []int{54} } func (m *CommandAddPartitionToTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5146,7 +5459,7 @@ func (m *Subscription) Reset() { *m = Subscription{} } func (m *Subscription) String() string { return proto.CompactTextString(m) } func (*Subscription) ProtoMessage() {} func (*Subscription) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{54} + return fileDescriptor_39529ba7ad9caeb8, []int{55} } func (m *Subscription) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5190,9 +5503,9 @@ func (m *Subscription) GetSubscription() string { } type CommandAddSubscriptionToTxn struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` Subscription []*Subscription `protobuf:"bytes,4,rep,name=subscription" json:"subscription,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -5203,7 +5516,7 @@ func (m *CommandAddSubscriptionToTxn) Reset() { *m = CommandAddSubscript func (m *CommandAddSubscriptionToTxn) String() string { return proto.CompactTextString(m) } func (*CommandAddSubscriptionToTxn) ProtoMessage() {} func (*CommandAddSubscriptionToTxn) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{55} + return fileDescriptor_39529ba7ad9caeb8, []int{56} } func (m *CommandAddSubscriptionToTxn) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5264,9 +5577,9 @@ func (m *CommandAddSubscriptionToTxn) GetSubscription() []*Subscription { } type CommandAddSubscriptionToTxnResponse struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` Error *ServerError `protobuf:"varint,4,opt,name=error,enum=pulsar.proto.ServerError" json:"error,omitempty"` Message *string `protobuf:"bytes,5,opt,name=message" json:"message,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -5278,7 +5591,7 @@ func (m *CommandAddSubscriptionToTxnResponse) Reset() { *m = CommandAddS func (m *CommandAddSubscriptionToTxnResponse) String() string { return proto.CompactTextString(m) } func (*CommandAddSubscriptionToTxnResponse) ProtoMessage() {} func (*CommandAddSubscriptionToTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{56} + return fileDescriptor_39529ba7ad9caeb8, []int{57} } func (m *CommandAddSubscriptionToTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5346,10 +5659,10 @@ func (m *CommandAddSubscriptionToTxnResponse) GetMessage() string { } type CommandEndTxn struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` - TxnAction *TxnAction `protobuf:"varint,4,opt,name=txn_action,enum=pulsar.proto.TxnAction" json:"txn_action,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` + TxnAction *TxnAction `protobuf:"varint,4,opt,name=txn_action,json=txnAction,enum=pulsar.proto.TxnAction" json:"txn_action,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -5359,7 +5672,7 @@ func (m *CommandEndTxn) Reset() { *m = CommandEndTxn{} } func (m *CommandEndTxn) String() string { return proto.CompactTextString(m) } func (*CommandEndTxn) ProtoMessage() {} func (*CommandEndTxn) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{57} + return fileDescriptor_39529ba7ad9caeb8, []int{58} } func (m *CommandEndTxn) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5420,9 +5733,9 @@ func (m *CommandEndTxn) GetTxnAction() TxnAction { } type CommandEndTxnResponse struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` Error *ServerError `protobuf:"varint,4,opt,name=error,enum=pulsar.proto.ServerError" json:"error,omitempty"` Message *string `protobuf:"bytes,5,opt,name=message" json:"message,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -5434,7 +5747,7 @@ func (m *CommandEndTxnResponse) Reset() { *m = CommandEndTxnResponse{} } func (m *CommandEndTxnResponse) String() string { return proto.CompactTextString(m) } func (*CommandEndTxnResponse) ProtoMessage() {} func (*CommandEndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{58} + return fileDescriptor_39529ba7ad9caeb8, []int{59} } func (m *CommandEndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5502,21 +5815,22 @@ func (m *CommandEndTxnResponse) GetMessage() string { } type CommandEndTxnOnPartition struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` - Topic *string `protobuf:"bytes,4,opt,name=topic" json:"topic,omitempty"` - TxnAction *TxnAction `protobuf:"varint,5,opt,name=txn_action,enum=pulsar.proto.TxnAction" json:"txn_action,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` + Topic *string `protobuf:"bytes,4,opt,name=topic" json:"topic,omitempty"` + TxnAction *TxnAction `protobuf:"varint,5,opt,name=txn_action,json=txnAction,enum=pulsar.proto.TxnAction" json:"txn_action,omitempty"` + TxnidLeastBitsOfLowWatermark *uint64 `protobuf:"varint,6,opt,name=txnid_least_bits_of_low_watermark,json=txnidLeastBitsOfLowWatermark" json:"txnid_least_bits_of_low_watermark,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *CommandEndTxnOnPartition) Reset() { *m = CommandEndTxnOnPartition{} } func (m *CommandEndTxnOnPartition) String() string { return proto.CompactTextString(m) } func (*CommandEndTxnOnPartition) ProtoMessage() {} func (*CommandEndTxnOnPartition) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{59} + return fileDescriptor_39529ba7ad9caeb8, []int{60} } func (m *CommandEndTxnOnPartition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5583,10 +5897,17 @@ func (m *CommandEndTxnOnPartition) GetTxnAction() TxnAction { return TxnAction_COMMIT } +func (m *CommandEndTxnOnPartition) GetTxnidLeastBitsOfLowWatermark() uint64 { + if m != nil && m.TxnidLeastBitsOfLowWatermark != nil { + return *m.TxnidLeastBitsOfLowWatermark + } + return 0 +} + type CommandEndTxnOnPartitionResponse struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` Error *ServerError `protobuf:"varint,4,opt,name=error,enum=pulsar.proto.ServerError" json:"error,omitempty"` Message *string `protobuf:"bytes,5,opt,name=message" json:"message,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -5598,7 +5919,7 @@ func (m *CommandEndTxnOnPartitionResponse) Reset() { *m = CommandEndTxnO func (m *CommandEndTxnOnPartitionResponse) String() string { return proto.CompactTextString(m) } func (*CommandEndTxnOnPartitionResponse) ProtoMessage() {} func (*CommandEndTxnOnPartitionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{60} + return fileDescriptor_39529ba7ad9caeb8, []int{61} } func (m *CommandEndTxnOnPartitionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5666,21 +5987,22 @@ func (m *CommandEndTxnOnPartitionResponse) GetMessage() string { } type CommandEndTxnOnSubscription struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` - Subscription *Subscription `protobuf:"bytes,4,opt,name=subscription" json:"subscription,omitempty"` - TxnAction *TxnAction `protobuf:"varint,5,opt,name=txn_action,enum=pulsar.proto.TxnAction" json:"txn_action,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` + Subscription *Subscription `protobuf:"bytes,4,opt,name=subscription" json:"subscription,omitempty"` + TxnAction *TxnAction `protobuf:"varint,5,opt,name=txn_action,json=txnAction,enum=pulsar.proto.TxnAction" json:"txn_action,omitempty"` + TxnidLeastBitsOfLowWatermark *uint64 `protobuf:"varint,6,opt,name=txnid_least_bits_of_low_watermark,json=txnidLeastBitsOfLowWatermark" json:"txnid_least_bits_of_low_watermark,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *CommandEndTxnOnSubscription) Reset() { *m = CommandEndTxnOnSubscription{} } func (m *CommandEndTxnOnSubscription) String() string { return proto.CompactTextString(m) } func (*CommandEndTxnOnSubscription) ProtoMessage() {} func (*CommandEndTxnOnSubscription) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{61} + return fileDescriptor_39529ba7ad9caeb8, []int{62} } func (m *CommandEndTxnOnSubscription) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5747,10 +6069,17 @@ func (m *CommandEndTxnOnSubscription) GetTxnAction() TxnAction { return TxnAction_COMMIT } +func (m *CommandEndTxnOnSubscription) GetTxnidLeastBitsOfLowWatermark() uint64 { + if m != nil && m.TxnidLeastBitsOfLowWatermark != nil { + return *m.TxnidLeastBitsOfLowWatermark + } + return 0 +} + type CommandEndTxnOnSubscriptionResponse struct { - RequestId *uint64 `protobuf:"varint,1,req,name=request_id" json:"request_id,omitempty"` - TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,def=0" json:"txnid_least_bits,omitempty"` - TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,def=0" json:"txnid_most_bits,omitempty"` + RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` + TxnidLeastBits *uint64 `protobuf:"varint,2,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` + TxnidMostBits *uint64 `protobuf:"varint,3,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` Error *ServerError `protobuf:"varint,4,opt,name=error,enum=pulsar.proto.ServerError" json:"error,omitempty"` Message *string `protobuf:"bytes,5,opt,name=message" json:"message,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -5762,7 +6091,7 @@ func (m *CommandEndTxnOnSubscriptionResponse) Reset() { *m = CommandEndT func (m *CommandEndTxnOnSubscriptionResponse) String() string { return proto.CompactTextString(m) } func (*CommandEndTxnOnSubscriptionResponse) ProtoMessage() {} func (*CommandEndTxnOnSubscriptionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{62} + return fileDescriptor_39529ba7ad9caeb8, []int{63} } func (m *CommandEndTxnOnSubscriptionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5836,17 +6165,17 @@ type BaseCommand struct { Subscribe *CommandSubscribe `protobuf:"bytes,4,opt,name=subscribe" json:"subscribe,omitempty"` Producer *CommandProducer `protobuf:"bytes,5,opt,name=producer" json:"producer,omitempty"` Send *CommandSend `protobuf:"bytes,6,opt,name=send" json:"send,omitempty"` - SendReceipt *CommandSendReceipt `protobuf:"bytes,7,opt,name=send_receipt" json:"send_receipt,omitempty"` - SendError *CommandSendError `protobuf:"bytes,8,opt,name=send_error" json:"send_error,omitempty"` + SendReceipt *CommandSendReceipt `protobuf:"bytes,7,opt,name=send_receipt,json=sendReceipt" json:"send_receipt,omitempty"` + SendError *CommandSendError `protobuf:"bytes,8,opt,name=send_error,json=sendError" json:"send_error,omitempty"` Message *CommandMessage `protobuf:"bytes,9,opt,name=message" json:"message,omitempty"` Ack *CommandAck `protobuf:"bytes,10,opt,name=ack" json:"ack,omitempty"` Flow *CommandFlow `protobuf:"bytes,11,opt,name=flow" json:"flow,omitempty"` Unsubscribe *CommandUnsubscribe `protobuf:"bytes,12,opt,name=unsubscribe" json:"unsubscribe,omitempty"` Success *CommandSuccess `protobuf:"bytes,13,opt,name=success" json:"success,omitempty"` Error *CommandError `protobuf:"bytes,14,opt,name=error" json:"error,omitempty"` - CloseProducer *CommandCloseProducer `protobuf:"bytes,15,opt,name=close_producer" json:"close_producer,omitempty"` - CloseConsumer *CommandCloseConsumer `protobuf:"bytes,16,opt,name=close_consumer" json:"close_consumer,omitempty"` - ProducerSuccess *CommandProducerSuccess `protobuf:"bytes,17,opt,name=producer_success" json:"producer_success,omitempty"` + CloseProducer *CommandCloseProducer `protobuf:"bytes,15,opt,name=close_producer,json=closeProducer" json:"close_producer,omitempty"` + CloseConsumer *CommandCloseConsumer `protobuf:"bytes,16,opt,name=close_consumer,json=closeConsumer" json:"close_consumer,omitempty"` + ProducerSuccess *CommandProducerSuccess `protobuf:"bytes,17,opt,name=producer_success,json=producerSuccess" json:"producer_success,omitempty"` Ping *CommandPing `protobuf:"bytes,18,opt,name=ping" json:"ping,omitempty"` Pong *CommandPong `protobuf:"bytes,19,opt,name=pong" json:"pong,omitempty"` RedeliverUnacknowledgedMessages *CommandRedeliverUnacknowledgedMessages `protobuf:"bytes,20,opt,name=redeliverUnacknowledgedMessages" json:"redeliverUnacknowledgedMessages,omitempty"` @@ -5860,7 +6189,7 @@ type BaseCommand struct { Seek *CommandSeek `protobuf:"bytes,28,opt,name=seek" json:"seek,omitempty"` GetLastMessageId *CommandGetLastMessageId `protobuf:"bytes,29,opt,name=getLastMessageId" json:"getLastMessageId,omitempty"` GetLastMessageIdResponse *CommandGetLastMessageIdResponse `protobuf:"bytes,30,opt,name=getLastMessageIdResponse" json:"getLastMessageIdResponse,omitempty"` - ActiveConsumerChange *CommandActiveConsumerChange `protobuf:"bytes,31,opt,name=active_consumer_change" json:"active_consumer_change,omitempty"` + ActiveConsumerChange *CommandActiveConsumerChange `protobuf:"bytes,31,opt,name=active_consumer_change,json=activeConsumerChange" json:"active_consumer_change,omitempty"` GetTopicsOfNamespace *CommandGetTopicsOfNamespace `protobuf:"bytes,32,opt,name=getTopicsOfNamespace" json:"getTopicsOfNamespace,omitempty"` GetTopicsOfNamespaceResponse *CommandGetTopicsOfNamespaceResponse `protobuf:"bytes,33,opt,name=getTopicsOfNamespaceResponse" json:"getTopicsOfNamespaceResponse,omitempty"` GetSchema *CommandGetSchema `protobuf:"bytes,34,opt,name=getSchema" json:"getSchema,omitempty"` @@ -5892,7 +6221,7 @@ func (m *BaseCommand) Reset() { *m = BaseCommand{} } func (m *BaseCommand) String() string { return proto.CompactTextString(m) } func (*BaseCommand) ProtoMessage() {} func (*BaseCommand) Descriptor() ([]byte, []int) { - return fileDescriptor_1370d448d7fffc03, []int{63} + return fileDescriptor_39529ba7ad9caeb8, []int{64} } func (m *BaseCommand) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6287,6 +6616,7 @@ func (m *BaseCommand) GetEndTxnOnSubscriptionResponse() *CommandEndTxnOnSubscrip func init() { proto.RegisterEnum("pulsar.proto.CompressionType", CompressionType_name, CompressionType_value) + proto.RegisterEnum("pulsar.proto.ProducerAccessMode", ProducerAccessMode_name, ProducerAccessMode_value) proto.RegisterEnum("pulsar.proto.ServerError", ServerError_name, ServerError_value) proto.RegisterEnum("pulsar.proto.AuthMethod", AuthMethod_name, AuthMethod_value) proto.RegisterEnum("pulsar.proto.ProtocolVersion", ProtocolVersion_name, ProtocolVersion_value) @@ -6309,6 +6639,7 @@ func init() { proto.RegisterType((*EncryptionKeys)(nil), "pulsar.proto.EncryptionKeys") proto.RegisterType((*MessageMetadata)(nil), "pulsar.proto.MessageMetadata") proto.RegisterType((*SingleMessageMetadata)(nil), "pulsar.proto.SingleMessageMetadata") + proto.RegisterType((*BrokerEntryMetadata)(nil), "pulsar.proto.BrokerEntryMetadata") proto.RegisterType((*CommandConnect)(nil), "pulsar.proto.CommandConnect") proto.RegisterType((*FeatureFlags)(nil), "pulsar.proto.FeatureFlags") proto.RegisterType((*CommandConnected)(nil), "pulsar.proto.CommandConnected") @@ -6367,319 +6698,396 @@ func init() { proto.RegisterType((*BaseCommand)(nil), "pulsar.proto.BaseCommand") } -func init() { - proto.RegisterFile("github.com/apache/pulsar-client-go@v0.3.0/pulsar/internal/pulsar_proto/PulsarApi.proto", fileDescriptor_1370d448d7fffc03) -} - -var fileDescriptor_1370d448d7fffc03 = []byte{ - // 4908 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x7b, 0xcb, 0x6f, 0x1b, 0x49, - 0x7a, 0xb8, 0x9b, 0x0f, 0x89, 0xfc, 0xf8, 0x2a, 0xb7, 0x5e, 0x6d, 0x5b, 0xe3, 0xa1, 0x7b, 0xfc, - 0xa0, 0x3d, 0x3f, 0xcb, 0xb6, 0xfc, 0x9a, 0xf1, 0xec, 0xfe, 0x76, 0x29, 0xaa, 0x2d, 0x73, 0x25, - 0x91, 0x1a, 0x92, 0xb2, 0x77, 0x17, 0x58, 0x34, 0x4a, 0xdd, 0x65, 0xaa, 0xa1, 0x66, 0x37, 0xd3, - 0xdd, 0x94, 0xad, 0xc9, 0x39, 0x08, 0x72, 0x0a, 0x10, 0x04, 0x58, 0x20, 0x48, 0xb0, 0x87, 0x1c, - 0x82, 0x04, 0xc8, 0x25, 0xd7, 0x00, 0xb9, 0xe4, 0x90, 0xe4, 0x10, 0x20, 0xff, 0x41, 0x82, 0x39, - 0xe4, 0x18, 0xe4, 0x9c, 0x4b, 0x82, 0xaa, 0xea, 0x17, 0xa9, 0x66, 0xcb, 0x9e, 0xec, 0x6e, 0xe6, - 0xc4, 0xee, 0xae, 0xaa, 0xaf, 0xbe, 0xf7, 0xab, 0x8a, 0xf0, 0x7a, 0x68, 0x78, 0xc7, 0x93, 0xa3, - 0x0d, 0xcd, 0x1e, 0x3d, 0xc0, 0x63, 0xac, 0x1d, 0x93, 0x07, 0xe3, 0x89, 0xe9, 0x62, 0xe7, 0xbe, - 0x66, 0x1a, 0xc4, 0xf2, 0xee, 0x0f, 0xed, 0x1f, 0x9f, 0x3e, 0xdc, 0x78, 0xbc, 0xf1, 0xd0, 0xff, - 0xfe, 0xc0, 0xb0, 0x3c, 0xe2, 0x58, 0xd8, 0xf4, 0xdf, 0xd5, 0xb1, 0x63, 0x7b, 0xf6, 0x83, 0x03, - 0xf6, 0xd2, 0x1c, 0x1b, 0x1b, 0xec, 0x5d, 0x2c, 0xf3, 0x51, 0xfe, 0x26, 0xff, 0x5d, 0x06, 0x16, - 0xfa, 0xda, 0x31, 0x19, 0x61, 0xb1, 0x0c, 0x39, 0x0b, 0x8f, 0x88, 0x24, 0xd4, 0x33, 0x8d, 0xa2, - 0xb8, 0x04, 0x25, 0x97, 0x7d, 0x57, 0x75, 0xec, 0x61, 0x29, 0x5b, 0xcf, 0x34, 0xca, 0xe2, 0x1d, - 0xc8, 0x79, 0x67, 0x63, 0x22, 0xe5, 0xea, 0x99, 0x46, 0x75, 0xf3, 0xca, 0x46, 0x1c, 0xd4, 0x06, - 0x07, 0xb3, 0x31, 0x38, 0x1b, 0x13, 0xf1, 0x1e, 0xc0, 0xd8, 0xb1, 0xc7, 0xc4, 0xf1, 0x0c, 0xe2, - 0x4a, 0xf9, 0x7a, 0xb6, 0x51, 0xda, 0x5c, 0x9d, 0x9e, 0xbe, 0x4b, 0xce, 0x5e, 0x63, 0x73, 0x42, - 0xe4, 0xbf, 0x15, 0x20, 0xc7, 0x16, 0x15, 0x20, 0xd7, 0xb1, 0x2d, 0x82, 0x2e, 0x89, 0x00, 0x0b, - 0x7d, 0xcf, 0x31, 0xac, 0x21, 0x12, 0xe8, 0xd7, 0x9f, 0xb8, 0xb6, 0x85, 0x32, 0x62, 0x19, 0x0a, - 0x07, 0x74, 0xe9, 0xd1, 0xe4, 0x2d, 0xca, 0xd2, 0xef, 0xcd, 0x53, 0xc7, 0x46, 0x39, 0xfa, 0xb4, - 0x65, 0xdb, 0x26, 0xca, 0xd3, 0xa7, 0xb6, 0xe5, 0x7d, 0x81, 0x16, 0xc4, 0x22, 0xe4, 0xdb, 0x96, - 0xf7, 0xe8, 0x19, 0x5a, 0xf4, 0x1f, 0x1f, 0x6f, 0xa2, 0x82, 0xff, 0xf8, 0xec, 0x09, 0x2a, 0xd2, - 0xc7, 0x97, 0xa6, 0x8d, 0x3d, 0x04, 0x74, 0xb7, 0x6d, 0x7b, 0x72, 0x64, 0x12, 0x54, 0xa2, 0x10, - 0xb6, 0xb1, 0x47, 0x50, 0x99, 0x3e, 0x0d, 0x8c, 0x11, 0x41, 0x15, 0xb1, 0x02, 0x45, 0xfa, 0xe4, - 0x7a, 0x78, 0x34, 0x46, 0x55, 0x8a, 0x46, 0x80, 0x3b, 0xaa, 0xc9, 0x47, 0x50, 0xd9, 0x27, 0xae, - 0x8b, 0x87, 0xa4, 0xad, 0x6f, 0x63, 0x0f, 0x8b, 0x08, 0x0a, 0x26, 0xd1, 0x87, 0xc4, 0x69, 0xeb, - 0x8c, 0x95, 0x39, 0xb1, 0x06, 0x8b, 0xc4, 0xf2, 0x9c, 0xb3, 0xb6, 0x2e, 0x65, 0xd8, 0x87, 0x15, - 0x28, 0x8e, 0xb1, 0xe3, 0x19, 0x9e, 0x61, 0x5b, 0x52, 0xb6, 0x2e, 0x34, 0xf2, 0x2f, 0x32, 0xf7, - 0x1f, 0x89, 0x6b, 0x50, 0x3a, 0xc2, 0x9e, 0x76, 0xac, 0x1a, 0x96, 0x4e, 0xde, 0x4b, 0xb9, 0x60, - 0x40, 0xbe, 0x1d, 0xed, 0x28, 0x96, 0x20, 0x7b, 0x42, 0xce, 0x7c, 0x21, 0x55, 0x20, 0x7f, 0x4a, - 0xbf, 0x32, 0xb8, 0x45, 0xf9, 0x1e, 0x94, 0x77, 0xc9, 0xd9, 0x9e, 0x6d, 0x0d, 0x2f, 0x9a, 0x9b, - 0xa3, 0x30, 0xdb, 0x96, 0xd7, 0xc3, 0xd6, 0x90, 0xd0, 0x21, 0xd7, 0xc3, 0x8e, 0xc7, 0x66, 0xe6, - 0xe9, 0x32, 0x62, 0x71, 0x5c, 0xf3, 0xf2, 0x6b, 0xa8, 0x2a, 0x96, 0xe6, 0x9c, 0x8d, 0x29, 0xb2, - 0xbb, 0xe4, 0xcc, 0x4d, 0x81, 0x5a, 0x16, 0x1b, 0x50, 0x18, 0x11, 0x0f, 0xfb, 0x2a, 0x93, 0x26, - 0xf5, 0x5f, 0xe5, 0xa1, 0xe6, 0x33, 0x6e, 0xdf, 0x5f, 0x21, 0xae, 0x40, 0x65, 0xec, 0xd8, 0xfa, - 0x44, 0x23, 0x8e, 0x3a, 0xa3, 0x8a, 0xe4, 0x77, 0x26, 0xc4, 0xd2, 0x88, 0x6a, 0x04, 0x3c, 0x5c, - 0x86, 0xf2, 0x78, 0x72, 0x64, 0x1a, 0xee, 0xb1, 0xea, 0x19, 0x23, 0xc2, 0x14, 0x34, 0x37, 0xa3, - 0x77, 0xb9, 0x34, 0x0c, 0xc4, 0x35, 0xa8, 0x39, 0x64, 0x6c, 0x1a, 0x1a, 0xf6, 0x88, 0xae, 0xbe, - 0x75, 0xec, 0x91, 0x94, 0xaf, 0x0b, 0x8d, 0x22, 0x43, 0x23, 0x10, 0x8f, 0x4a, 0x49, 0x5d, 0x60, - 0x9f, 0x97, 0xa1, 0x1c, 0xce, 0x57, 0x3d, 0x5b, 0x5a, 0xac, 0x67, 0x1b, 0x45, 0xf1, 0x0b, 0x28, - 0x69, 0xf6, 0x68, 0xec, 0x10, 0xd7, 0xa5, 0xd2, 0x2c, 0xd4, 0x85, 0x46, 0x75, 0xf3, 0x93, 0xe9, - 0x2d, 0x5b, 0xd1, 0x04, 0xaa, 0xe8, 0x2f, 0x72, 0x9d, 0x6e, 0x47, 0x11, 0xd7, 0xe1, 0xf2, 0xc4, - 0x0a, 0xd6, 0x12, 0x5d, 0x75, 0x8d, 0x6f, 0x88, 0x54, 0xac, 0x0b, 0x8d, 0xca, 0x0b, 0xe1, 0xa1, - 0x58, 0x87, 0x15, 0x6b, 0x32, 0x52, 0x47, 0x9c, 0x45, 0xae, 0x6a, 0x58, 0x2a, 0x53, 0x0e, 0xa9, - 0xc4, 0xd4, 0x42, 0x78, 0x24, 0xae, 0x00, 0x90, 0x53, 0x62, 0x79, 0x9c, 0xfe, 0x72, 0x5d, 0x68, - 0xe4, 0xe8, 0xc2, 0xa7, 0x50, 0x23, 0xa1, 0xc0, 0x28, 0xfa, 0xae, 0x54, 0x61, 0x7c, 0x58, 0x9f, - 0x46, 0x6a, 0x46, 0xaa, 0x6b, 0x53, 0xcb, 0xb0, 0x39, 0xb4, 0xa5, 0x2a, 0x23, 0x5b, 0x02, 0x14, - 0x1b, 0x18, 0x63, 0x07, 0x8f, 0xa4, 0x5a, 0x5d, 0x68, 0x94, 0xc5, 0x55, 0xa8, 0xfa, 0x2e, 0xe2, - 0x94, 0x38, 0x8c, 0x7a, 0xc4, 0xbe, 0x37, 0xe0, 0xca, 0x14, 0xff, 0xd4, 0xa3, 0x67, 0x4f, 0x54, - 0x62, 0x69, 0xb6, 0x4e, 0x74, 0xe9, 0x72, 0x5d, 0x68, 0x14, 0x5e, 0xe4, 0xdf, 0x62, 0xd3, 0x25, - 0x94, 0xa5, 0xb6, 0xa3, 0x13, 0x6a, 0xe9, 0x8c, 0xd1, 0x22, 0x5b, 0xbf, 0x06, 0x35, 0x9d, 0x98, - 0xc6, 0x29, 0x71, 0x54, 0xec, 0x53, 0xb7, 0x54, 0x17, 0x1a, 0x59, 0xaa, 0x08, 0x23, 0xec, 0x9c, - 0x10, 0x47, 0x65, 0x5e, 0x68, 0x99, 0x72, 0x42, 0xbc, 0x06, 0xc8, 0x7b, 0x6f, 0x19, 0xba, 0x6a, - 0x12, 0xec, 0x7a, 0xea, 0x91, 0xe1, 0xb9, 0xd2, 0x6a, 0xc0, 0x8c, 0xab, 0x50, 0xe3, 0x83, 0x23, - 0x3b, 0x18, 0x5b, 0x0b, 0xc6, 0xae, 0xc3, 0xd2, 0xb1, 0x31, 0x3c, 0x26, 0xae, 0xa7, 0xc6, 0xd5, - 0x4b, 0xf2, 0xc7, 0xe5, 0xff, 0x12, 0x60, 0xa5, 0x6f, 0x58, 0x43, 0x93, 0xcc, 0xea, 0xe9, 0xb4, - 0x96, 0x09, 0xa9, 0x5a, 0x76, 0x4e, 0x99, 0x32, 0x81, 0x32, 0x8d, 0xf1, 0x99, 0x69, 0x63, 0x5f, - 0xee, 0x59, 0x66, 0x79, 0xeb, 0x50, 0xa1, 0x0a, 0x81, 0x35, 0xaa, 0x91, 0xf6, 0xc4, 0x63, 0x3e, - 0x20, 0xe4, 0xd6, 0xb4, 0xc0, 0xf3, 0x01, 0x1d, 0xa9, 0xec, 0x5e, 0x48, 0x63, 0xf7, 0x22, 0x63, - 0xf7, 0x8c, 0x79, 0x51, 0x0d, 0xce, 0xc9, 0xff, 0x90, 0x81, 0x6a, 0xcb, 0x1e, 0x8d, 0xb0, 0xa5, - 0xb7, 0x6c, 0xcb, 0x22, 0x9a, 0x47, 0xc5, 0xcd, 0x23, 0x4f, 0x28, 0x6e, 0x6e, 0x9e, 0xf7, 0xa1, - 0x84, 0x27, 0xde, 0xb1, 0x3a, 0x22, 0xde, 0xb1, 0xad, 0x33, 0xfa, 0xaa, 0x9b, 0xd2, 0x34, 0x3b, - 0x9a, 0x13, 0xef, 0x78, 0x9f, 0x8d, 0x53, 0x7d, 0x8a, 0x4d, 0xe7, 0x76, 0xce, 0xed, 0xee, 0x32, - 0x14, 0xd9, 0x88, 0xef, 0x3d, 0x28, 0x6e, 0xd7, 0x00, 0x31, 0x00, 0x9a, 0x6d, 0x86, 0xbb, 0x72, - 0xbf, 0x28, 0x3c, 0x14, 0xaf, 0xc1, 0xd2, 0xd8, 0xb1, 0xdf, 0x9f, 0xa9, 0x9e, 0xad, 0x1e, 0x39, - 0x36, 0xd5, 0x8b, 0x89, 0x63, 0xfa, 0xd6, 0x7a, 0x15, 0x44, 0xdb, 0x31, 0x86, 0x86, 0x85, 0x4d, - 0x75, 0xec, 0x18, 0x96, 0x66, 0x8c, 0xb1, 0xc9, 0x28, 0x9e, 0x1e, 0x8b, 0x76, 0x2c, 0xb0, 0xb1, - 0x75, 0x58, 0x9e, 0x1e, 0xf3, 0xc9, 0x2a, 0xb2, 0xd1, 0x47, 0x50, 0x79, 0x4b, 0xb0, 0x37, 0x71, - 0x88, 0xfa, 0xd6, 0xc4, 0x43, 0x57, 0x82, 0xba, 0xd0, 0x28, 0x6d, 0x5e, 0x9d, 0xa6, 0xf6, 0x25, - 0x9f, 0xf2, 0x92, 0xce, 0x90, 0x9f, 0x40, 0x39, 0xfe, 0x2e, 0xde, 0x84, 0x15, 0x77, 0x32, 0x1e, - 0xdb, 0x8e, 0xe7, 0xf2, 0x0d, 0x1c, 0xf2, 0xd6, 0x21, 0xee, 0xb1, 0x24, 0xc4, 0x44, 0x25, 0x63, - 0x40, 0xd3, 0xec, 0x27, 0x3a, 0xb3, 0x37, 0xe2, 0x50, 0xb3, 0x98, 0x16, 0x40, 0x12, 0x93, 0x32, - 0x01, 0x93, 0x24, 0x40, 0x23, 0xfc, 0x3e, 0xf0, 0x23, 0x81, 0xb2, 0x09, 0x8d, 0xbc, 0xec, 0xc1, - 0x92, 0xbf, 0x05, 0x95, 0x4e, 0x8f, 0xb8, 0x63, 0xdb, 0x72, 0x49, 0xa2, 0x98, 0x29, 0xe9, 0x0d, - 0x28, 0x38, 0xfe, 0x1c, 0x06, 0xfd, 0x9c, 0xca, 0x53, 0x28, 0x2c, 0x02, 0x26, 0xe1, 0x93, 0xf5, - 0xf1, 0x91, 0x4f, 0x61, 0x39, 0xb6, 0x6b, 0xeb, 0x18, 0x9b, 0x26, 0xa1, 0x31, 0x28, 0x89, 0x38, - 0xba, 0xed, 0x5d, 0x28, 0x6a, 0xc1, 0xa4, 0xff, 0xcd, 0xbe, 0xcf, 0xa1, 0x10, 0x4e, 0x4c, 0x52, - 0x41, 0xe1, 0xbc, 0x0a, 0xd2, 0xdd, 0xca, 0xf2, 0x2f, 0x05, 0xa8, 0xec, 0x92, 0xb3, 0xfe, 0x31, - 0x76, 0x88, 0x4e, 0x5d, 0x80, 0xb8, 0x09, 0x95, 0x93, 0xf0, 0x83, 0xad, 0xf3, 0x30, 0x55, 0xdd, - 0xbc, 0x76, 0xce, 0x03, 0x44, 0x53, 0xa8, 0xcb, 0x38, 0xc6, 0xee, 0x31, 0x8b, 0xb7, 0x6e, 0x72, - 0x68, 0x0c, 0xc3, 0xf1, 0x6d, 0x58, 0xc3, 0xa6, 0x69, 0xbf, 0xeb, 0x4e, 0xbc, 0xee, 0xdb, 0x2e, - 0x35, 0xd8, 0x6d, 0xee, 0x0e, 0xcf, 0xa6, 0xfc, 0x81, 0xfc, 0x1f, 0xf9, 0x50, 0x49, 0xfa, 0x93, - 0x23, 0x57, 0x73, 0x8c, 0x23, 0x16, 0xcb, 0x3d, 0x7b, 0x6c, 0x68, 0xbe, 0x6e, 0x2c, 0x43, 0xd9, - 0xe5, 0x63, 0xcc, 0x7f, 0xf3, 0x44, 0x41, 0x7c, 0x0e, 0x8b, 0xee, 0xe4, 0x88, 0xc6, 0x22, 0xe6, - 0x78, 0xaa, 0x9b, 0xb7, 0xcf, 0x05, 0xac, 0x29, 0xa8, 0x1b, 0x7d, 0x3e, 0x9b, 0xfa, 0x0a, 0xcd, - 0xb6, 0xdc, 0xc9, 0x88, 0x38, 0xd4, 0x57, 0xe4, 0x58, 0xd0, 0x15, 0x01, 0x1c, 0xea, 0x40, 0x5c, - 0x8f, 0x7e, 0xcb, 0xfb, 0x29, 0x4e, 0x25, 0x9c, 0xc8, 0xf8, 0xcb, 0xad, 0x72, 0x15, 0xaa, 0x63, - 0xc7, 0xb0, 0x1d, 0xc3, 0x3b, 0x53, 0x4d, 0x72, 0x4a, 0xb8, 0x45, 0xe6, 0xc5, 0x15, 0x58, 0xd4, - 0x27, 0x0e, 0x3e, 0x32, 0x09, 0x33, 0xc3, 0xc2, 0x8b, 0x9c, 0xe7, 0x4c, 0x88, 0xf8, 0x14, 0x10, - 0x4b, 0x4c, 0x42, 0xf5, 0x35, 0xb8, 0x21, 0x96, 0x66, 0x99, 0x3d, 0x9d, 0x82, 0xc5, 0xb3, 0x10, - 0x48, 0xf5, 0xce, 0xab, 0x50, 0x75, 0x08, 0xd6, 0xd5, 0xd0, 0xeb, 0xb2, 0xf0, 0x5a, 0x10, 0x6f, - 0xc2, 0x02, 0x0f, 0x6d, 0x2c, 0xae, 0x96, 0x36, 0x97, 0x93, 0x52, 0x5d, 0xb1, 0x0b, 0x35, 0xc3, - 0x32, 0x3c, 0x03, 0x9b, 0x07, 0xb6, 0xcb, 0xb3, 0xb9, 0x0a, 0xf3, 0x7e, 0x1b, 0x17, 0xb0, 0xb3, - 0x3d, 0xbd, 0xea, 0xc5, 0xc2, 0x1e, 0xf6, 0x88, 0xeb, 0x89, 0x37, 0x61, 0x3d, 0x4a, 0x31, 0xe2, - 0x72, 0x53, 0x5d, 0x0f, 0x7b, 0x84, 0x45, 0xe4, 0x82, 0x28, 0xc3, 0xf2, 0x5b, 0xdb, 0xd1, 0x68, - 0x12, 0x32, 0x36, 0x34, 0x55, 0x73, 0x08, 0x66, 0x7b, 0xd7, 0x62, 0x9c, 0xdb, 0x80, 0xcf, 0xa6, - 0x39, 0xe7, 0xd8, 0xa6, 0x79, 0x84, 0xb5, 0x13, 0x95, 0xf2, 0x99, 0x83, 0x24, 0x1a, 0x0b, 0xd8, - 0x2c, 0x88, 0x4c, 0xe9, 0x34, 0xf1, 0x30, 0x8b, 0xd3, 0xa5, 0xf9, 0x3a, 0x4d, 0x3c, 0x2c, 0x6f, - 0xc1, 0x62, 0xa0, 0x17, 0x15, 0x28, 0x2a, 0xef, 0x35, 0x73, 0xe2, 0x1a, 0xa7, 0x41, 0xfe, 0xce, - 0xe6, 0x21, 0x81, 0xa6, 0xcb, 0x2f, 0xb1, 0x61, 0xda, 0xa7, 0xc4, 0x41, 0x19, 0xb1, 0x0a, 0xb0, - 0x4b, 0xce, 0x54, 0x7f, 0x34, 0x2b, 0x7f, 0x0e, 0xb5, 0x19, 0x66, 0xd0, 0xc5, 0x9c, 0x1d, 0xe8, - 0x12, 0x5d, 0xac, 0x60, 0xc7, 0x34, 0xe8, 0x9b, 0x20, 0xff, 0x89, 0x00, 0x9f, 0xfa, 0xbc, 0x3c, - 0x08, 0x22, 0x1e, 0xd1, 0x07, 0x94, 0x0b, 0x61, 0x6c, 0x9e, 0xd1, 0xff, 0x69, 0xdd, 0xe4, 0xa9, - 0x63, 0x72, 0x68, 0xc8, 0xa6, 0x84, 0x86, 0x5c, 0x6a, 0x68, 0x60, 0xd1, 0x4b, 0xfe, 0xfd, 0x0c, - 0xdc, 0xb9, 0x00, 0xb9, 0xd0, 0xc7, 0x8a, 0x00, 0x61, 0xc8, 0x76, 0x99, 0xeb, 0xa9, 0x24, 0x62, - 0xda, 0x8b, 0xf9, 0xdc, 0x2c, 0xd3, 0xac, 0x1f, 0x27, 0x6a, 0xd6, 0x45, 0x1b, 0x6e, 0xec, 0xd9, - 0xf6, 0xc9, 0x64, 0xcc, 0x44, 0xd5, 0x80, 0x3c, 0x71, 0x1c, 0xdb, 0x61, 0x44, 0x9d, 0x2f, 0xe2, - 0x98, 0x0f, 0x56, 0xe8, 0x04, 0x5a, 0xb7, 0xf8, 0xda, 0xe3, 0x93, 0x78, 0x0b, 0x20, 0x06, 0xa8, - 0x44, 0xc5, 0xaf, 0x69, 0xc4, 0x75, 0xb9, 0xc4, 0xa9, 0x94, 0xa9, 0xc4, 0xe5, 0x7f, 0x16, 0x40, - 0xf4, 0x11, 0xf3, 0xa7, 0x53, 0x9c, 0x3e, 0x44, 0x32, 0xeb, 0x50, 0xa1, 0x8c, 0xa5, 0xfe, 0x01, - 0x7b, 0xc6, 0x29, 0x27, 0x3a, 0x4c, 0x5f, 0x92, 0xe5, 0x96, 0x4b, 0x91, 0x5b, 0x3e, 0x55, 0x6e, - 0xdc, 0x25, 0xd5, 0x41, 0xc2, 0xfa, 0x29, 0xcd, 0xe5, 0x68, 0x12, 0x6e, 0x1a, 0xae, 0x47, 0xac, - 0xc0, 0x69, 0xb1, 0x74, 0x41, 0xfe, 0xf7, 0x0c, 0x5c, 0x3d, 0x4f, 0x4f, 0x28, 0x4c, 0x09, 0x10, - 0xcf, 0x3e, 0x28, 0xef, 0x0c, 0x8d, 0x1c, 0x3a, 0xa6, 0x1f, 0x4d, 0xae, 0xc1, 0xd2, 0xec, 0xc8, - 0xc0, 0x74, 0xfd, 0x0c, 0x50, 0x39, 0x27, 0xdb, 0xc7, 0x89, 0xb2, 0x4d, 0xd8, 0x32, 0x2e, 0xce, - 0x69, 0x36, 0xe6, 0x92, 0xd9, 0x98, 0x8f, 0xb3, 0x31, 0x54, 0x80, 0x85, 0x8f, 0x50, 0x80, 0x45, - 0x3f, 0x07, 0xb8, 0xe2, 0x67, 0x5c, 0xc7, 0x8e, 0x3d, 0x19, 0x1e, 0xab, 0x2e, 0x27, 0x8c, 0xe5, - 0x5d, 0x85, 0x78, 0x6c, 0x7a, 0x3c, 0xa5, 0x2a, 0x65, 0x28, 0xf4, 0x88, 0x6e, 0x38, 0x44, 0xa3, - 0x46, 0x5d, 0x82, 0x45, 0x3f, 0xa9, 0x41, 0x42, 0x4c, 0x71, 0x32, 0xd4, 0x84, 0x6a, 0x81, 0x46, - 0xfb, 0x75, 0xe0, 0xac, 0xd6, 0x2c, 0x41, 0x29, 0x2c, 0x11, 0x43, 0xb5, 0x99, 0xe6, 0x41, 0x36, - 0x08, 0x40, 0xd3, 0xb5, 0x64, 0xce, 0xaf, 0x66, 0x8a, 0x7e, 0x35, 0x43, 0xf4, 0x59, 0xb6, 0x44, - 0x41, 0x63, 0x21, 0x35, 0x68, 0x44, 0xc1, 0x61, 0x31, 0x25, 0x38, 0x20, 0xc8, 0x93, 0xb1, 0xad, - 0x1d, 0xf3, 0x84, 0x9a, 0xfa, 0xd8, 0xbb, 0x70, 0x6d, 0xe2, 0x12, 0xd6, 0x97, 0x39, 0x35, 0x74, - 0xa2, 0xab, 0xd3, 0x08, 0x16, 0x23, 0xf7, 0x2d, 0xff, 0xa5, 0x00, 0xa5, 0x20, 0x6a, 0x10, 0x4b, - 0x9f, 0x25, 0x9b, 0xf7, 0x15, 0x12, 0xeb, 0xe2, 0x35, 0x28, 0xc7, 0xeb, 0xc6, 0x20, 0x01, 0x7a, - 0x94, 0x58, 0x27, 0xe5, 0x52, 0xea, 0xa4, 0xfc, 0x05, 0x75, 0xd2, 0x42, 0x50, 0x27, 0xfd, 0x61, - 0x64, 0xee, 0x14, 0xd7, 0x1e, 0xd1, 0x88, 0x31, 0xf6, 0x3e, 0x02, 0xe5, 0x07, 0x00, 0xb1, 0xf8, - 0x9e, 0xbd, 0x38, 0xbe, 0xcf, 0xc1, 0x28, 0xa0, 0x46, 0x7e, 0x17, 0xe5, 0x45, 0xc4, 0xd2, 0xb9, - 0x2e, 0x7f, 0x38, 0x3a, 0xa1, 0x7d, 0x64, 0x13, 0xbb, 0x5c, 0xc9, 0xf6, 0x91, 0x63, 0x0d, 0x18, - 0x37, 0x2c, 0x9a, 0x7c, 0x84, 0x67, 0x13, 0x26, 0x21, 0x81, 0x60, 0xba, 0xeb, 0x05, 0x04, 0x5f, - 0x03, 0xe4, 0x10, 0xbf, 0x26, 0x3e, 0x53, 0x35, 0x7b, 0x62, 0x79, 0x8c, 0x4f, 0x15, 0x4a, 0xed, - 0x7f, 0x66, 0x01, 0x82, 0x94, 0x5a, 0x3b, 0x49, 0xde, 0x71, 0x13, 0x0a, 0x34, 0xf6, 0xb3, 0xb2, - 0x39, 0xc3, 0xc8, 0xaa, 0x27, 0x3a, 0x9b, 0xa6, 0x76, 0xb2, 0xd1, 0xd4, 0x4e, 0x98, 0xd1, 0xce, - 0x8a, 0x25, 0x7b, 0x11, 0x96, 0x5b, 0x80, 0x4e, 0xb1, 0x69, 0xe8, 0x3c, 0xbd, 0x88, 0x07, 0x99, - 0xc6, 0xdc, 0xcd, 0x5e, 0x87, 0x0b, 0x38, 0x4b, 0x37, 0x12, 0x1a, 0x87, 0x57, 0xcf, 0xd9, 0x61, - 0xd4, 0xe2, 0x4a, 0xd2, 0xea, 0x85, 0x14, 0xad, 0x5e, 0x0c, 0x74, 0xe4, 0x2e, 0x2c, 0x06, 0x84, - 0x56, 0x01, 0xda, 0x96, 0x6e, 0x9c, 0x1a, 0xfa, 0x04, 0x9b, 0xe8, 0x12, 0x7d, 0x6f, 0x4d, 0x46, - 0x13, 0x93, 0xf9, 0x4e, 0x24, 0x50, 0x05, 0xaf, 0xcd, 0xe2, 0x79, 0x1d, 0xae, 0x1e, 0xc6, 0x9a, - 0x37, 0x7d, 0xe3, 0x1b, 0xd2, 0xb2, 0x1d, 0x67, 0xc2, 0xb2, 0x35, 0x74, 0x49, 0x5c, 0x05, 0x71, - 0x9b, 0xc4, 0x1a, 0x43, 0x6c, 0x15, 0x12, 0xc4, 0x65, 0x40, 0xad, 0x63, 0xa2, 0x9d, 0xb8, 0x93, - 0xd1, 0xbe, 0xe1, 0x8e, 0xb0, 0xa7, 0x1d, 0xa3, 0x8c, 0x78, 0x05, 0x56, 0xb6, 0xe8, 0xe3, 0x36, - 0xe9, 0x13, 0xc7, 0xc0, 0xa6, 0xf1, 0x0d, 0xe1, 0x0b, 0xb2, 0xe2, 0x12, 0xd4, 0xb6, 0x49, 0xd0, - 0x7e, 0xe1, 0x1f, 0x73, 0xf2, 0x9f, 0x45, 0x26, 0xd7, 0xd4, 0x4e, 0xc2, 0x48, 0x94, 0x28, 0xfa, - 0x24, 0x0e, 0x65, 0x52, 0x38, 0x94, 0x8d, 0xfa, 0x0a, 0xdf, 0x39, 0x51, 0xd8, 0x83, 0x6b, 0x21, - 0x7a, 0x94, 0x89, 0x2d, 0x1f, 0xab, 0xd6, 0x31, 0x2b, 0x70, 0x12, 0xf1, 0x94, 0xa0, 0x68, 0xb8, - 0x2a, 0x66, 0xf3, 0x19, 0x82, 0x61, 0x2c, 0x79, 0x11, 0xfa, 0xc2, 0x97, 0xa6, 0xfd, 0x2e, 0x79, - 0xf5, 0x2a, 0x54, 0x7d, 0x14, 0x0e, 0x88, 0x33, 0xe2, 0x34, 0x66, 0x1a, 0x15, 0xf9, 0x87, 0x21, - 0xa3, 0x0e, 0x2d, 0x37, 0x2c, 0x92, 0x12, 0x41, 0x24, 0x24, 0x24, 0xf2, 0xef, 0xc5, 0xfd, 0x30, - 0x39, 0xf9, 0xe0, 0x85, 0x1f, 0xef, 0xd3, 0xd6, 0x61, 0x39, 0x58, 0x30, 0xd5, 0xd7, 0x64, 0x4e, - 0x4d, 0x7e, 0x00, 0x92, 0x8f, 0x46, 0x8f, 0x60, 0xed, 0x98, 0xe8, 0x8a, 0xa5, 0x77, 0xdf, 0xf2, - 0xbc, 0x2a, 0x09, 0x27, 0xf9, 0x47, 0x61, 0x99, 0xdd, 0x32, 0x6d, 0x97, 0x84, 0xe1, 0x34, 0xd1, - 0x0d, 0x26, 0x51, 0x3e, 0x03, 0x20, 0x90, 0xe0, 0x87, 0xb3, 0xce, 0x86, 0xdb, 0x21, 0xca, 0xbe, - 0xeb, 0x3a, 0xb4, 0xb0, 0x76, 0x62, 0xd9, 0xef, 0x58, 0x7f, 0x3c, 0x70, 0x91, 0x6e, 0x32, 0xc8, - 0x87, 0x50, 0x8a, 0x18, 0x48, 0xa5, 0x79, 0x91, 0xfb, 0x91, 0x7f, 0x12, 0x3a, 0x5f, 0x3f, 0x2d, - 0x9d, 0x41, 0x8b, 0xc3, 0x8d, 0x82, 0x77, 0x66, 0x7e, 0xf0, 0x96, 0xcf, 0x60, 0x75, 0x26, 0x11, - 0x49, 0x83, 0x79, 0x2e, 0xd7, 0xe0, 0x55, 0xf6, 0x3a, 0x20, 0x13, 0xcf, 0xc4, 0x28, 0xaa, 0x09, - 0x59, 0xd6, 0xed, 0x3f, 0xdf, 0x3d, 0xcd, 0xb1, 0x7e, 0xc3, 0x2f, 0xa0, 0xec, 0x6f, 0xcd, 0x8d, - 0x2b, 0x69, 0xc3, 0xd0, 0x34, 0x33, 0x1f, 0x11, 0xa2, 0xb2, 0x2c, 0x44, 0x55, 0x42, 0x85, 0x3e, - 0x30, 0xac, 0x61, 0xfc, 0xd5, 0xb6, 0x86, 0x71, 0xa9, 0xfb, 0x12, 0xe9, 0x7b, 0xd8, 0x4b, 0xa6, - 0x3a, 0xa9, 0x17, 0x20, 0xff, 0x51, 0x16, 0xd6, 0x93, 0x20, 0xc4, 0x4b, 0x9f, 0x73, 0x90, 0xee, - 0x03, 0x30, 0x72, 0x54, 0xcd, 0xd6, 0x89, 0xdf, 0x40, 0x4c, 0xa1, 0x69, 0x05, 0x2a, 0x7c, 0x7a, - 0x44, 0x99, 0xc0, 0x0b, 0x8a, 0x91, 0x3b, 0xec, 0x61, 0x8f, 0x74, 0xfd, 0xce, 0xa9, 0xc0, 0xba, - 0x5f, 0xee, 0x70, 0xc0, 0xb3, 0xd5, 0xf1, 0xc4, 0xa3, 0x23, 0xf9, 0xd8, 0x08, 0x9d, 0x1d, 0xaa, - 0x27, 0x0b, 0x1c, 0xd4, 0x45, 0x97, 0x03, 0xba, 0x3a, 0x61, 0x11, 0xc0, 0x7a, 0x46, 0xa7, 0xd8, - 0x30, 0xf1, 0x91, 0x19, 0xba, 0x18, 0x96, 0xd9, 0x89, 0x6b, 0x50, 0x9b, 0x50, 0xc5, 0x8e, 0x34, - 0x9a, 0xa5, 0x72, 0x54, 0xd5, 0xd6, 0x8f, 0x4c, 0x9b, 0x0e, 0x04, 0xac, 0xe8, 0x5a, 0x87, 0xfe, - 0x4c, 0xd7, 0xef, 0x1d, 0x16, 0xa8, 0x84, 0xb0, 0xae, 0xd3, 0x30, 0xc1, 0x7a, 0x0f, 0xac, 0x47, - 0xa2, 0x05, 0x3d, 0xbf, 0xbe, 0x61, 0x69, 0xbc, 0xb7, 0x5f, 0x14, 0xcb, 0xfe, 0xe1, 0x5b, 0x25, - 0x98, 0xe5, 0xe3, 0xaf, 0xbc, 0x1f, 0x1b, 0x0e, 0xd1, 0x59, 0x73, 0x40, 0xf0, 0xb9, 0xb0, 0x85, - 0xb5, 0x13, 0xd3, 0x1e, 0xb2, 0x96, 0x40, 0x4e, 0xde, 0x82, 0x35, 0x5f, 0x26, 0x3b, 0xc4, 0xdb, - 0xc3, 0xae, 0x17, 0x1a, 0xce, 0x87, 0x9b, 0xf3, 0x49, 0x58, 0x7a, 0xcf, 0xc2, 0x08, 0x45, 0xfb, - 0x04, 0x6a, 0x4c, 0xdf, 0x63, 0x8e, 0x4f, 0xb8, 0x38, 0xb7, 0x49, 0xda, 0xec, 0xef, 0x85, 0x30, - 0x80, 0xec, 0x10, 0x8f, 0xb9, 0x39, 0xb7, 0xfb, 0x96, 0x4a, 0xc3, 0x1d, 0x63, 0x2d, 0x59, 0x89, - 0x2e, 0x43, 0xd1, 0x0a, 0x26, 0xf8, 0x06, 0xf8, 0x0a, 0x72, 0x23, 0xaa, 0x51, 0xd9, 0x94, 0xa6, - 0x4c, 0x12, 0xfc, 0x8d, 0x7d, 0x5b, 0x27, 0x2f, 0xe0, 0x40, 0xe9, 0xf5, 0xdb, 0xfd, 0x81, 0xd2, - 0x19, 0xc8, 0x8f, 0x21, 0xc7, 0xda, 0x78, 0x55, 0x88, 0x7d, 0x45, 0x97, 0x44, 0x11, 0xaa, 0x9d, - 0x6e, 0x47, 0x8d, 0x7d, 0x13, 0xc4, 0x45, 0xc8, 0x36, 0xf7, 0xf6, 0x50, 0x46, 0x6e, 0xc3, 0x67, - 0x29, 0x9b, 0xa4, 0x5a, 0x44, 0x15, 0x16, 0x58, 0xd5, 0xc3, 0x1d, 0x5f, 0x51, 0xde, 0x0f, 0x33, - 0xda, 0x1d, 0xe2, 0xf9, 0x05, 0x46, 0xd2, 0xba, 0xb0, 0x5a, 0xe2, 0x0c, 0x38, 0xef, 0x63, 0x58, - 0x5b, 0x5d, 0xfe, 0x1b, 0x21, 0x8c, 0x27, 0x21, 0xbc, 0xdf, 0x82, 0x85, 0x46, 0xbe, 0x37, 0x97, - 0x52, 0x38, 0x9d, 0x47, 0x3a, 0xcf, 0x90, 0xd6, 0xe2, 0x38, 0x77, 0x9d, 0x96, 0x43, 0xb0, 0x47, - 0x3e, 0x9c, 0x17, 0xd1, 0xe6, 0x59, 0xa6, 0x94, 0xc9, 0x8e, 0xff, 0x8f, 0x05, 0xa8, 0xcf, 0xdb, - 0xe5, 0xb7, 0xc0, 0xa1, 0x79, 0x41, 0xe1, 0x6b, 0xa8, 0xf8, 0x58, 0x75, 0xc8, 0xbb, 0xc1, 0x7b, - 0x2b, 0x11, 0x05, 0x9e, 0xcc, 0xa9, 0x9e, 0x67, 0xaa, 0x2e, 0xd1, 0x6c, 0x4b, 0x8f, 0x25, 0x7a, - 0x08, 0xf2, 0x9e, 0x16, 0x04, 0x20, 0x96, 0x00, 0xff, 0x4a, 0x80, 0x95, 0x29, 0x98, 0xa9, 0xe4, - 0xfd, 0x5f, 0x64, 0x91, 0xbf, 0x1b, 0x0a, 0xbc, 0xa9, 0x47, 0x2d, 0xae, 0x81, 0x3d, 0x8f, 0xfe, - 0xef, 0x8c, 0xe3, 0x74, 0x3b, 0x2e, 0xc7, 0x2c, 0xee, 0xaf, 0x22, 0x45, 0x38, 0xb7, 0xfb, 0xf7, - 0x8e, 0x53, 0x8f, 0xa1, 0xdc, 0x8f, 0x75, 0x8b, 0x3f, 0xe8, 0x10, 0x40, 0xfe, 0xd3, 0xc8, 0xc9, - 0x36, 0x75, 0x3d, 0xbe, 0xfe, 0x37, 0xc0, 0xe2, 0x87, 0x33, 0x28, 0xe4, 0x92, 0x2a, 0xbb, 0x38, - 0x0e, 0xf2, 0x5f, 0x0b, 0xa1, 0xfb, 0x4c, 0x42, 0xef, 0x7b, 0x27, 0x83, 0x3f, 0x10, 0x42, 0x1b, - 0x55, 0x2c, 0xfd, 0xd7, 0xce, 0xc0, 0xcf, 0x01, 0xa8, 0x71, 0xd3, 0xfa, 0xc8, 0xf7, 0x0a, 0xd5, - 0xcd, 0xb5, 0x69, 0xf4, 0x06, 0xef, 0xad, 0x26, 0x1b, 0x8e, 0xdb, 0x36, 0xc7, 0xe5, 0x7b, 0xc7, - 0xad, 0x3f, 0x8f, 0x22, 0x10, 0xc7, 0xb0, 0x6b, 0x85, 0x36, 0xf6, 0xeb, 0x45, 0x32, 0xb4, 0x05, - 0xde, 0x00, 0x9c, 0xe6, 0x63, 0x3e, 0x9d, 0x8f, 0x31, 0x27, 0x70, 0x0e, 0xcb, 0xef, 0x1d, 0x4b, - 0xff, 0x29, 0xb2, 0xe7, 0x00, 0xd9, 0x29, 0xa7, 0xf0, 0x1b, 0xb6, 0x67, 0x21, 0xdd, 0x9e, 0x3f, - 0x8e, 0xf1, 0x31, 0xe3, 0x4f, 0xa2, 0xe5, 0x7b, 0xc7, 0xfb, 0xff, 0xbe, 0x01, 0xa5, 0x2d, 0x4c, - 0xcb, 0x64, 0x86, 0xb3, 0xf8, 0xff, 0xfc, 0x5c, 0x9d, 0x9f, 0x0c, 0x5f, 0x9f, 0x86, 0x14, 0x9b, - 0xc8, 0x6f, 0xcb, 0xdd, 0x87, 0x45, 0x3f, 0xe3, 0xf7, 0x8b, 0xd2, 0xf5, 0xc4, 0x54, 0x35, 0xb8, - 0x88, 0xf1, 0x08, 0x8a, 0x61, 0x81, 0xe0, 0xb7, 0x16, 0xae, 0xa7, 0x2d, 0x20, 0x3a, 0x5d, 0x12, - 0x76, 0x3f, 0x7c, 0x59, 0x5d, 0x4f, 0x3f, 0xa3, 0x14, 0x1f, 0x40, 0x21, 0x28, 0x6a, 0x19, 0x91, - 0xa5, 0x84, 0x5b, 0x4d, 0x53, 0x9d, 0xfa, 0x3b, 0x90, 0x73, 0x89, 0xc5, 0x1b, 0xc3, 0xa5, 0x59, - 0xee, 0xc5, 0x9b, 0xd9, 0xcf, 0xa0, 0x4c, 0x27, 0xaa, 0x0e, 0xef, 0x14, 0xfb, 0x5d, 0xf4, 0xfa, - 0xdc, 0x05, 0x41, 0x47, 0x79, 0x13, 0x80, 0xad, 0xe3, 0x42, 0x2a, 0xa4, 0x51, 0x11, 0xb6, 0x7d, - 0xef, 0x47, 0x92, 0x2a, 0xa6, 0xb0, 0x36, 0x68, 0xd7, 0xde, 0x82, 0x2c, 0xd6, 0x4e, 0xfc, 0x5b, - 0x1d, 0xd2, 0xbc, 0xae, 0x25, 0x25, 0xf5, 0xad, 0x69, 0xbf, 0x63, 0x05, 0xdb, 0x3c, 0x52, 0x59, - 0xaf, 0xea, 0x29, 0x94, 0x26, 0x51, 0xdf, 0xc9, 0x3f, 0x4c, 0x4e, 0xa6, 0x34, 0xde, 0x9f, 0xba, - 0x0f, 0x8b, 0x2e, 0xef, 0x37, 0xb0, 0x6a, 0x6f, 0x1e, 0xd6, 0x41, 0x4f, 0xe2, 0x6e, 0xa0, 0xb8, - 0xd5, 0x24, 0x2b, 0x9c, 0xea, 0x26, 0xbc, 0x80, 0xaa, 0x66, 0xda, 0x2e, 0x09, 0xcf, 0x1e, 0x58, - 0x89, 0x58, 0xda, 0x94, 0x93, 0x15, 0x68, 0xaa, 0x77, 0x14, 0xae, 0x0d, 0x2a, 0x46, 0x76, 0x7c, - 0x9c, 0xba, 0x36, 0x6c, 0x1b, 0xfd, 0x7f, 0x76, 0x37, 0x83, 0xb7, 0x48, 0x02, 0xd2, 0xf8, 0x11, - 0xf3, 0xcd, 0x54, 0xad, 0x0a, 0x48, 0xbc, 0x03, 0xb9, 0xb1, 0x61, 0x0d, 0xd9, 0x0d, 0xb1, 0x79, - 0x1c, 0x3f, 0x30, 0xac, 0x21, 0x9b, 0x68, 0x5b, 0x43, 0x76, 0x63, 0x6c, 0xee, 0x44, 0xdb, 0x1a, - 0x8a, 0xbf, 0x80, 0x4f, 0x9d, 0xf4, 0xc6, 0x14, 0xbb, 0x60, 0x56, 0xda, 0x7c, 0x92, 0x08, 0xe3, - 0xa2, 0xa6, 0xd6, 0x2b, 0xb8, 0x1c, 0xe6, 0x94, 0xc1, 0x71, 0xac, 0xb4, 0xc2, 0x00, 0xde, 0xff, - 0xa8, 0x33, 0x5c, 0xf1, 0xa7, 0xb1, 0xfb, 0x5d, 0xb3, 0x07, 0xbb, 0xec, 0xa6, 0x5b, 0x69, 0xf3, - 0xe9, 0x77, 0x3a, 0x15, 0xa6, 0xda, 0x69, 0x46, 0xa7, 0x8b, 0xec, 0x66, 0xdc, 0x3c, 0xed, 0x8c, - 0x1f, 0xe4, 0x2a, 0xb0, 0x64, 0x9e, 0x3f, 0x94, 0x64, 0x17, 0xe7, 0x4a, 0x73, 0x5a, 0xfd, 0x49, - 0xe7, 0xa6, 0x5f, 0x46, 0x57, 0x44, 0x58, 0x8b, 0x48, 0xba, 0x92, 0xa6, 0x4d, 0x53, 0xed, 0xa8, - 0x36, 0xac, 0x68, 0x49, 0xdd, 0x25, 0xe9, 0x2a, 0x03, 0x71, 0xef, 0x62, 0x10, 0x21, 0x16, 0x4d, - 0xb8, 0xec, 0xcc, 0xb6, 0x54, 0xa5, 0x6b, 0x0c, 0xcc, 0xed, 0x39, 0x82, 0x9f, 0x6d, 0xc0, 0x32, - 0xc7, 0x47, 0x4e, 0xa4, 0xf5, 0x54, 0xc7, 0x47, 0x4e, 0xc4, 0x1f, 0x01, 0x1a, 0xce, 0x34, 0x4f, - 0xa4, 0x4f, 0xd8, 0xa2, 0x5b, 0xf3, 0x7a, 0x13, 0xd3, 0xdd, 0x9a, 0x2e, 0x48, 0xc3, 0x39, 0xdd, - 0x17, 0xe9, 0x7a, 0x8a, 0x6e, 0xcd, 0x6d, 0xd9, 0xb4, 0x61, 0x95, 0x77, 0xdc, 0x43, 0x9b, 0x56, - 0x35, 0xd6, 0xa3, 0x97, 0x3e, 0x65, 0xe0, 0xee, 0xce, 0x71, 0x81, 0x09, 0x4d, 0xfd, 0x1d, 0x58, - 0x1e, 0x26, 0xb4, 0x39, 0xa4, 0x7a, 0x0a, 0xa0, 0xc4, 0xe6, 0xce, 0x1b, 0x58, 0x1f, 0xa6, 0xf4, - 0x4b, 0xa4, 0x1b, 0x0c, 0xe0, 0xa3, 0x0f, 0x06, 0x18, 0x12, 0xfb, 0x08, 0x8a, 0xc3, 0xa0, 0xdb, - 0x21, 0xc9, 0x29, 0xe1, 0x23, 0xea, 0xb1, 0x34, 0xe1, 0xf2, 0x70, 0xb6, 0x41, 0x22, 0x7d, 0x96, - 0xa2, 0x1d, 0xe7, 0xdb, 0x29, 0x5f, 0xf2, 0xc3, 0xf8, 0xf0, 0xa6, 0x9b, 0x74, 0x33, 0x45, 0xcd, - 0xa7, 0xef, 0xc4, 0x3d, 0x87, 0x32, 0x8e, 0x5d, 0xcd, 0x93, 0x6e, 0xb1, 0x95, 0x37, 0xe6, 0xae, - 0x8c, 0x1b, 0x36, 0x8e, 0xce, 0x85, 0xa4, 0xdb, 0x29, 0x86, 0x1d, 0x3f, 0x3f, 0xe2, 0xd4, 0x4e, - 0x37, 0x3d, 0xa4, 0x3b, 0xe9, 0xd4, 0xce, 0x34, 0x62, 0xbe, 0x86, 0x2b, 0xc3, 0x79, 0x7d, 0x13, - 0xa9, 0xc1, 0x40, 0x6d, 0x7c, 0x18, 0xa8, 0x10, 0xab, 0xcf, 0x61, 0xc1, 0x62, 0x0d, 0x0a, 0x69, - 0x33, 0xe9, 0x18, 0x65, 0xba, 0x2f, 0xf2, 0x15, 0x54, 0xad, 0xa9, 0x6e, 0x86, 0xf4, 0x98, 0x2d, - 0xfa, 0x2c, 0x65, 0x51, 0x9c, 0x7e, 0x3c, 0x5b, 0xeb, 0x4b, 0x4f, 0x52, 0xe8, 0x3f, 0xdf, 0x97, - 0xf8, 0x1a, 0xae, 0xe0, 0x79, 0xed, 0x02, 0xe9, 0x69, 0x0a, 0xfd, 0xf3, 0x9b, 0x0c, 0x3b, 0xb0, - 0x8c, 0x13, 0x0a, 0x60, 0xe9, 0x59, 0x9a, 0x85, 0x26, 0x15, 0xf4, 0x6f, 0x60, 0x1d, 0xa7, 0x54, - 0xd2, 0xd2, 0xf3, 0x14, 0xc3, 0x4a, 0x2d, 0xc1, 0x3f, 0x87, 0x05, 0xc2, 0xb2, 0x74, 0xe9, 0x8b, - 0x14, 0x09, 0xf9, 0x55, 0xf1, 0x57, 0x50, 0x25, 0x53, 0x35, 0xa9, 0xf4, 0x65, 0x8a, 0x84, 0x66, - 0xca, 0xd7, 0x26, 0x5c, 0x26, 0xb3, 0x85, 0x98, 0xf4, 0x22, 0x45, 0x42, 0xe7, 0x8b, 0xcb, 0xaf, - 0xe1, 0x0a, 0x99, 0x57, 0xcb, 0x49, 0x5f, 0xa5, 0x48, 0x68, 0x7e, 0x05, 0xb8, 0x03, 0xcb, 0x24, - 0xa1, 0x4a, 0x91, 0x7e, 0x90, 0x22, 0xa1, 0xc4, 0x12, 0xed, 0x0d, 0xac, 0x93, 0x94, 0x72, 0x47, - 0xfa, 0x61, 0x8a, 0x84, 0xd2, 0xea, 0x24, 0xf9, 0x5f, 0x0b, 0xfe, 0x7f, 0x6c, 0x4a, 0xb0, 0xd8, - 0xea, 0x76, 0x3a, 0x4a, 0x6b, 0x80, 0x32, 0x62, 0x05, 0x8a, 0xfe, 0x8b, 0xb2, 0x8d, 0xb2, 0xf4, - 0xb5, 0x7f, 0xb8, 0xd5, 0x6f, 0xf5, 0xda, 0x5b, 0x0a, 0xca, 0xb1, 0xbf, 0xdb, 0xf4, 0xba, 0xdb, - 0x87, 0x2d, 0xa5, 0xc7, 0xff, 0x5a, 0xd3, 0x57, 0x3a, 0xdb, 0x68, 0x41, 0x44, 0x50, 0xa6, 0x4f, - 0x6a, 0x4f, 0x69, 0x29, 0xed, 0x83, 0x01, 0x5a, 0x14, 0xab, 0x00, 0xec, 0x8b, 0xd2, 0xeb, 0x75, - 0x7b, 0xa8, 0x40, 0x37, 0xd9, 0x57, 0xfa, 0xfd, 0xe6, 0x8e, 0x82, 0x8a, 0xac, 0x1d, 0xde, 0xda, - 0x45, 0x40, 0x21, 0xbc, 0xdc, 0xeb, 0xbe, 0x41, 0x25, 0xb1, 0x06, 0xa5, 0xc3, 0x4e, 0xb4, 0x55, - 0x99, 0x5d, 0x25, 0x3b, 0x6c, 0xb5, 0x94, 0x7e, 0x1f, 0x55, 0xc4, 0x22, 0xe4, 0x39, 0xa0, 0xaa, - 0x28, 0x42, 0xb5, 0xb5, 0xd7, 0xed, 0x2b, 0x6a, 0x88, 0x48, 0x2d, 0xfa, 0xd6, 0xea, 0x76, 0xfa, - 0x87, 0xfb, 0x4a, 0x0f, 0x21, 0x71, 0x19, 0x50, 0x30, 0x43, 0x0d, 0x00, 0x5d, 0xa6, 0x1b, 0x1e, - 0xb4, 0x3b, 0x3b, 0x48, 0x64, 0x4f, 0xdd, 0xce, 0x0e, 0x5a, 0x12, 0x6f, 0xc1, 0x8d, 0x9e, 0xb2, - 0xad, 0xec, 0xb5, 0x5f, 0x2b, 0x3d, 0xf5, 0xb0, 0xd3, 0x6c, 0xed, 0x76, 0xba, 0x6f, 0xf6, 0x94, - 0xed, 0x1d, 0x65, 0x5b, 0xf5, 0x71, 0xee, 0xa3, 0x65, 0x51, 0x82, 0xe5, 0x83, 0x66, 0x6f, 0xd0, - 0x1e, 0xb4, 0xbb, 0x1d, 0x36, 0x32, 0x68, 0x6e, 0x37, 0x07, 0x4d, 0xb4, 0x22, 0xde, 0x80, 0x4f, - 0x92, 0x46, 0xd4, 0x9e, 0xd2, 0x3f, 0xe8, 0x76, 0xfa, 0x0a, 0x5a, 0x65, 0x17, 0x18, 0xbb, 0xdd, - 0xdd, 0xc3, 0x03, 0xb4, 0x26, 0x2e, 0x41, 0x8d, 0x3f, 0x47, 0x13, 0x24, 0x46, 0x82, 0x8f, 0xbc, - 0xda, 0x1f, 0x34, 0x07, 0x7d, 0x74, 0x45, 0xbc, 0x06, 0x6b, 0xd3, 0xdf, 0xa2, 0x05, 0x57, 0x29, - 0x3a, 0x3d, 0xa5, 0xd9, 0x7a, 0xa5, 0x6c, 0xab, 0x94, 0xcf, 0xdd, 0x97, 0xea, 0xa0, 0x7b, 0xd0, - 0x6e, 0xa1, 0x6b, 0x5c, 0x2c, 0xca, 0x2e, 0x5a, 0x17, 0xd7, 0x60, 0x69, 0x47, 0x19, 0xa8, 0x7b, - 0xcd, 0xfe, 0x20, 0xa0, 0x44, 0x6d, 0x6f, 0xa3, 0x4f, 0xc4, 0x3a, 0xac, 0x27, 0x0c, 0x44, 0xe0, - 0xaf, 0x8b, 0x57, 0x61, 0xb5, 0xd9, 0x1a, 0xb4, 0x5f, 0x47, 0x3c, 0x55, 0x5b, 0xaf, 0x9a, 0x9d, - 0x1d, 0x05, 0x7d, 0x4a, 0xf1, 0xa2, 0xab, 0xd9, 0x7e, 0x7d, 0xba, 0x73, 0xa7, 0xb9, 0xaf, 0xf4, - 0x0f, 0x9a, 0x2d, 0x05, 0xd5, 0xc5, 0x9b, 0x50, 0x9f, 0x33, 0x18, 0x81, 0xbf, 0x41, 0xd5, 0x83, - 0xce, 0xea, 0xb7, 0x5e, 0x29, 0xfb, 0x4d, 0x24, 0x07, 0x98, 0xf2, 0xf7, 0x68, 0xe2, 0x67, 0x94, - 0x2f, 0xcd, 0xc3, 0xc1, 0x2b, 0xba, 0xf9, 0xde, 0x9e, 0x42, 0xf7, 0xbf, 0x29, 0x5e, 0x86, 0x0a, - 0xfb, 0x16, 0x4e, 0xbb, 0x45, 0x15, 0xb0, 0xd9, 0xda, 0x8d, 0xbe, 0xdc, 0xa6, 0xfc, 0xa1, 0x10, - 0xbb, 0x3d, 0xb5, 0xd5, 0x53, 0x9a, 0x03, 0x25, 0xd8, 0xeb, 0x0e, 0x15, 0x57, 0xd2, 0x48, 0xb4, - 0xb8, 0x41, 0x95, 0xaf, 0xa3, 0xbc, 0x51, 0x07, 0x3f, 0xed, 0xa0, 0x4d, 0xaa, 0x49, 0xfe, 0x4b, - 0x34, 0xe5, 0x31, 0x85, 0xdf, 0xdc, 0xde, 0x56, 0x43, 0xc1, 0xab, 0x83, 0x2e, 0x9b, 0xff, 0x84, - 0xc2, 0x4f, 0x1a, 0x89, 0x16, 0x3f, 0xa5, 0x1c, 0xa4, 0x53, 0x7c, 0x7d, 0x3f, 0x88, 0xaf, 0x7f, - 0x46, 0x39, 0x38, 0x67, 0x30, 0x02, 0xf1, 0x9c, 0xa2, 0x48, 0xe5, 0x4e, 0x97, 0x7c, 0x41, 0x51, - 0xf4, 0x5f, 0xa2, 0x29, 0x5f, 0x52, 0x14, 0x83, 0xaf, 0xdd, 0x4e, 0x84, 0x0f, 0x7a, 0x41, 0x51, - 0x4c, 0x1a, 0x89, 0x16, 0x7f, 0x45, 0x51, 0x8c, 0x4d, 0x89, 0x23, 0x83, 0x7e, 0x40, 0x51, 0x9c, - 0x33, 0x18, 0x81, 0xf8, 0xe1, 0xbd, 0x6d, 0x76, 0x75, 0x2f, 0xfe, 0x37, 0x27, 0xf6, 0x7f, 0xbe, - 0x6e, 0x47, 0x41, 0x97, 0xa8, 0x0f, 0xd8, 0xfb, 0xf9, 0x13, 0xfe, 0x67, 0xbe, 0x9f, 0xef, 0xb5, - 0xb7, 0x50, 0x86, 0x3d, 0xf5, 0x07, 0xd4, 0xed, 0x00, 0x2c, 0xf4, 0x3b, 0xcd, 0x83, 0x83, 0x9f, - 0xa1, 0xdc, 0xbd, 0xbf, 0xc8, 0x41, 0x29, 0xde, 0x68, 0x41, 0x50, 0x3e, 0xb4, 0x68, 0x75, 0xe5, - 0x5f, 0x7d, 0xb9, 0x44, 0xf5, 0x21, 0xa8, 0x63, 0x62, 0x77, 0x6a, 0x0e, 0x88, 0xe3, 0xb2, 0x9b, - 0x9b, 0x9a, 0x7f, 0x71, 0x26, 0x43, 0xb5, 0x8c, 0xe6, 0x44, 0xc4, 0xf2, 0x0c, 0x2d, 0xba, 0xb8, - 0x83, 0xb2, 0xe2, 0x2a, 0x88, 0x4d, 0x7e, 0x3b, 0xf2, 0x9b, 0xd8, 0xf7, 0x1c, 0xdd, 0x2b, 0x48, - 0x69, 0xb7, 0x26, 0xee, 0x19, 0xca, 0x53, 0xe3, 0xf5, 0x6f, 0x6e, 0x76, 0x6c, 0xaf, 0x47, 0xb0, - 0x7e, 0x86, 0x16, 0xa8, 0x07, 0x09, 0x8a, 0xd3, 0x2d, 0x7e, 0xba, 0xfb, 0xf5, 0xc4, 0xf6, 0xb0, - 0xf2, 0x5e, 0x23, 0x44, 0x27, 0xbc, 0xcc, 0x46, 0x8b, 0xe2, 0x5d, 0xb8, 0x95, 0x3a, 0xed, 0xbd, - 0x46, 0xf8, 0x5d, 0xa1, 0x02, 0x25, 0x29, 0xb8, 0x13, 0xc4, 0x57, 0x17, 0xa9, 0x40, 0x0e, 0x2d, - 0xff, 0x6f, 0x22, 0x44, 0x7f, 0xcd, 0xcf, 0x82, 0xf8, 0x20, 0xd0, 0xf9, 0x2c, 0xc9, 0xed, 0xd8, - 0xde, 0x4b, 0x7b, 0x62, 0xe9, 0xa8, 0x44, 0xa5, 0x1f, 0xf7, 0xfb, 0xe1, 0x48, 0x99, 0x5d, 0x38, - 0x0a, 0x4e, 0xb3, 0x83, 0xaf, 0x15, 0x4a, 0xd9, 0xc0, 0xb6, 0xf7, 0xb1, 0x75, 0xd6, 0xe3, 0xed, - 0x33, 0x17, 0x55, 0x29, 0x10, 0x06, 0x77, 0x40, 0x9c, 0x91, 0x61, 0x61, 0x2f, 0x20, 0xa6, 0x46, - 0x59, 0x13, 0x12, 0x43, 0x59, 0xc3, 0x3c, 0x6e, 0xdb, 0x62, 0xb7, 0xbd, 0x38, 0x2a, 0x78, 0x44, - 0xd0, 0x65, 0xca, 0xda, 0x36, 0xbb, 0x15, 0x85, 0x3d, 0xe3, 0xc8, 0xf4, 0x13, 0x39, 0x24, 0x52, - 0x59, 0x04, 0x48, 0x34, 0x5d, 0xd7, 0x18, 0xfa, 0xa4, 0x2c, 0x89, 0x32, 0x5c, 0x1f, 0x38, 0xd8, - 0x72, 0x79, 0x57, 0xb0, 0x65, 0xdb, 0x8e, 0x4e, 0x77, 0xb6, 0x23, 0x5c, 0x97, 0xe3, 0x5b, 0xbd, - 0xb7, 0x68, 0xf1, 0x36, 0x71, 0xd1, 0xca, 0xbd, 0x5d, 0x80, 0xd8, 0x9f, 0x8a, 0xa8, 0xe7, 0x08, - 0xdf, 0xfc, 0x3f, 0x91, 0x2e, 0x41, 0x2d, 0xfa, 0xf6, 0x33, 0x0d, 0xbf, 0x7e, 0xc4, 0x75, 0x25, - 0xfa, 0xd8, 0xa4, 0xea, 0xe1, 0xa2, 0xcc, 0xbd, 0x5f, 0x0a, 0x50, 0x3b, 0xf0, 0xff, 0x3a, 0xe2, - 0x33, 0x5b, 0x5c, 0x80, 0xcc, 0xe9, 0x43, 0x74, 0x89, 0xfd, 0xd2, 0x95, 0xf4, 0x77, 0x13, 0x65, - 0xd8, 0xef, 0x63, 0x94, 0x65, 0xbf, 0x4f, 0x50, 0x8e, 0xfd, 0x3e, 0x45, 0x79, 0xf6, 0xfb, 0x0c, - 0x2d, 0xb0, 0xdf, 0xe7, 0x68, 0x91, 0xfd, 0x7e, 0x81, 0x0a, 0xec, 0xf7, 0x4b, 0x1e, 0x07, 0x4f, - 0x1f, 0x3d, 0x44, 0xc0, 0x1f, 0x1e, 0xa1, 0x12, 0x7f, 0xd8, 0x44, 0x65, 0xfe, 0xf0, 0x18, 0x55, - 0xf8, 0xc3, 0x13, 0x54, 0xe5, 0x0f, 0x4f, 0x51, 0xed, 0xde, 0xe7, 0xf1, 0x3f, 0x9f, 0xf8, 0x27, - 0xd0, 0xcd, 0xc3, 0x41, 0x57, 0xed, 0x1f, 0xec, 0xb5, 0x07, 0xfe, 0x55, 0xfb, 0x41, 0xbb, 0xb5, - 0xfb, 0x33, 0x24, 0xdc, 0x93, 0xa1, 0x18, 0xb6, 0x50, 0xe9, 0x40, 0xab, 0xbb, 0xbf, 0xcf, 0x26, - 0x15, 0x21, 0xdf, 0xdc, 0xea, 0xf6, 0x06, 0x48, 0xd8, 0xda, 0xfc, 0xc7, 0x6f, 0xaf, 0x0b, 0xff, - 0xf2, 0xed, 0x75, 0xe1, 0xdf, 0xbe, 0xbd, 0x2e, 0x80, 0x6c, 0x3b, 0xc3, 0x0d, 0xfe, 0xef, 0xe2, - 0x20, 0xa9, 0xd0, 0xec, 0xd1, 0xc8, 0xb6, 0x36, 0x70, 0xf0, 0x87, 0xe1, 0x57, 0xd9, 0xff, 0x09, - 0x00, 0x00, 0xff, 0xff, 0xc5, 0x98, 0x27, 0x47, 0x8b, 0x3c, 0x00, 0x00, +func init() { proto.RegisterFile("PulsarApi.proto", fileDescriptor_39529ba7ad9caeb8) } + +var fileDescriptor_39529ba7ad9caeb8 = []byte{ + // 6163 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x3c, 0x4b, 0x70, 0x1b, 0x47, + 0x76, 0x1a, 0x00, 0x24, 0x81, 0x07, 0x02, 0x6c, 0x35, 0x29, 0x69, 0xf4, 0x31, 0x45, 0x8d, 0x2c, + 0x9b, 0x96, 0x2d, 0xad, 0x44, 0xc9, 0xb2, 0x2d, 0x7b, 0xb3, 0x06, 0x41, 0x48, 0xc2, 0x92, 0x04, + 0xb8, 0x03, 0x50, 0x8e, 0x9d, 0xdd, 0xcc, 0x0e, 0x67, 0x9a, 0xe0, 0x14, 0x07, 0x33, 0xd8, 0x99, + 0x01, 0x25, 0xba, 0x2a, 0x39, 0xe4, 0x92, 0x53, 0x2a, 0x95, 0x43, 0xae, 0x49, 0x25, 0xa7, 0xe4, + 0x98, 0xaa, 0x3d, 0xa4, 0x2a, 0x55, 0xc9, 0x25, 0x9f, 0xad, 0xca, 0x25, 0xa9, 0x4a, 0x0e, 0x7b, + 0xda, 0xd4, 0x56, 0x3e, 0x87, 0xad, 0xa4, 0x72, 0xcb, 0x31, 0xa9, 0xfe, 0xcc, 0x0f, 0x18, 0x00, + 0x94, 0xbd, 0x29, 0xbb, 0x7c, 0xc2, 0xcc, 0xeb, 0xd7, 0x6f, 0x5e, 0xbf, 0xf7, 0xfa, 0xbd, 0xd7, + 0xaf, 0xbb, 0x01, 0x4b, 0x7b, 0x43, 0xdb, 0xd7, 0xbd, 0xda, 0xc0, 0xba, 0x3b, 0xf0, 0xdc, 0xc0, + 0xc5, 0x8b, 0x03, 0x06, 0xe0, 0x6f, 0xca, 0x8f, 0xf3, 0x30, 0xdf, 0x31, 0x8e, 0x48, 0x5f, 0xc7, + 0x18, 0x0a, 0x8e, 0xde, 0x27, 0xb2, 0xb4, 0x96, 0x5b, 0x2f, 0xa9, 0xec, 0x19, 0x5f, 0x87, 0xb2, + 0xcf, 0x5a, 0x35, 0x53, 0x0f, 0x74, 0x39, 0xbf, 0x96, 0x5b, 0x5f, 0x54, 0x81, 0x83, 0xb6, 0xf4, + 0x40, 0xc7, 0x77, 0xa0, 0x10, 0x9c, 0x0e, 0x88, 0x5c, 0x58, 0xcb, 0xad, 0x57, 0x37, 0x2e, 0xdf, + 0x4d, 0x12, 0xbf, 0xcb, 0x09, 0xdf, 0xed, 0x9e, 0x0e, 0x88, 0xca, 0xd0, 0xf0, 0x23, 0x80, 0x81, + 0xe7, 0x0e, 0x88, 0x17, 0x58, 0xc4, 0x97, 0xe7, 0xd6, 0xf2, 0xeb, 0xe5, 0x8d, 0x8b, 0xe9, 0x4e, + 0xdb, 0xe4, 0xf4, 0xb9, 0x6e, 0x0f, 0x89, 0x9a, 0xc0, 0x54, 0x7e, 0x27, 0x07, 0x05, 0x4a, 0x06, + 0x17, 0xa1, 0xd0, 0x72, 0x1d, 0x82, 0xce, 0x61, 0x80, 0xf9, 0x4e, 0xe0, 0x59, 0x4e, 0x0f, 0x49, + 0x14, 0xfa, 0x5d, 0xdf, 0x75, 0x50, 0x0e, 0x2f, 0x42, 0x71, 0x8f, 0x92, 0x39, 0x18, 0x1e, 0xa2, + 0x3c, 0x85, 0xd7, 0x4e, 0x3c, 0x17, 0x15, 0xe8, 0xd3, 0xa6, 0xeb, 0xda, 0x68, 0x8e, 0x3e, 0x35, + 0x9d, 0xe0, 0x7d, 0x34, 0x8f, 0x4b, 0x30, 0xd7, 0x74, 0x82, 0xfb, 0x8f, 0xd0, 0x82, 0x78, 0x7c, + 0xb0, 0x81, 0x8a, 0xe2, 0xf1, 0xd1, 0x43, 0x54, 0xa2, 0x8f, 0x4f, 0x6c, 0x57, 0x0f, 0x10, 0xd0, + 0xaf, 0x6d, 0xb9, 0xc3, 0x03, 0x9b, 0xa0, 0x32, 0xa5, 0xb0, 0xa5, 0x07, 0x04, 0x2d, 0xd2, 0xa7, + 0xae, 0xd5, 0x27, 0xa8, 0x82, 0x2b, 0x50, 0xa2, 0x4f, 0x7e, 0xa0, 0xf7, 0x07, 0xa8, 0x4a, 0xd9, + 0x08, 0xc7, 0x81, 0x96, 0x70, 0x19, 0x16, 0x9a, 0x8e, 0x1f, 0xe8, 0x4e, 0x80, 0x10, 0xc5, 0xdc, + 0x71, 0x0d, 0xdd, 0x66, 0x24, 0xce, 0x47, 0xaf, 0x8c, 0x0e, 0xc6, 0xe7, 0xa1, 0x12, 0xb5, 0x32, + 0xd0, 0x32, 0xc6, 0x50, 0x0d, 0x87, 0xd4, 0xd2, 0x03, 0xeb, 0x84, 0xa0, 0x15, 0xe5, 0xaf, 0x25, + 0xa8, 0xec, 0x12, 0xdf, 0xd7, 0x7b, 0xa4, 0x69, 0x32, 0x45, 0x5c, 0x81, 0xa2, 0x4d, 0xcc, 0x1e, + 0xf1, 0x9a, 0x26, 0xd3, 0x60, 0x41, 0x8d, 0xde, 0xb1, 0x0c, 0x0b, 0xc4, 0x09, 0xbc, 0xd3, 0xa6, + 0x29, 0xe7, 0x58, 0x53, 0xf8, 0x8a, 0xd7, 0xa0, 0x34, 0xd0, 0xbd, 0xc0, 0x0a, 0x2c, 0xd7, 0x91, + 0xf3, 0x6b, 0xd2, 0xfa, 0xdc, 0xe3, 0xdc, 0x9d, 0xfb, 0x6a, 0x0c, 0xc4, 0x37, 0xa1, 0x7c, 0xa0, + 0x07, 0xc6, 0x91, 0x66, 0x39, 0x26, 0x79, 0x29, 0x17, 0x22, 0x1c, 0x60, 0xe0, 0x26, 0x85, 0xe2, + 0x4b, 0xb0, 0xa0, 0x1b, 0xc7, 0x9a, 0x4f, 0x02, 0xa6, 0xd3, 0xbc, 0x3a, 0xaf, 0x1b, 0xc7, 0x1d, + 0x12, 0xe0, 0xd7, 0x80, 0xa3, 0x69, 0xbe, 0xf5, 0x39, 0x91, 0xe7, 0x69, 0x67, 0xb5, 0xc4, 0x20, + 0x1d, 0xeb, 0x73, 0xa2, 0x6c, 0xc4, 0x62, 0xc2, 0x08, 0xf2, 0xc7, 0xe4, 0x54, 0x58, 0x1f, 0x7d, + 0xc4, 0x2b, 0x30, 0x77, 0x42, 0x9b, 0x18, 0xd3, 0x25, 0x95, 0xbf, 0x28, 0x8f, 0x60, 0x71, 0x9b, + 0x9c, 0xee, 0xb8, 0x4e, 0xef, 0x4c, 0xfd, 0x0a, 0x61, 0xbf, 0x0d, 0x28, 0x36, 0x9d, 0x40, 0xd5, + 0x9d, 0x1e, 0xa1, 0x18, 0x7e, 0xa0, 0x7b, 0x01, 0xeb, 0x35, 0xa7, 0xf2, 0x17, 0x4a, 0x89, 0x38, + 0x5c, 0x44, 0x73, 0x2a, 0x7d, 0x54, 0x6c, 0xa8, 0x36, 0x1c, 0xc3, 0x3b, 0x1d, 0x50, 0x51, 0x6c, + 0x93, 0x53, 0x7f, 0xd6, 0xd7, 0x16, 0xc5, 0xd7, 0xf0, 0x06, 0x14, 0xfb, 0x24, 0xd0, 0xc5, 0xac, + 0x99, 0x66, 0xe6, 0x11, 0x9e, 0xf2, 0x8f, 0x25, 0x58, 0x12, 0x4a, 0xdd, 0x15, 0x30, 0x7c, 0x13, + 0x2a, 0x03, 0xcf, 0x35, 0x87, 0x06, 0xf1, 0xb4, 0xc4, 0xec, 0x5c, 0x0c, 0x81, 0xad, 0x70, 0x96, + 0x92, 0x1f, 0x0d, 0x89, 0x63, 0x10, 0xcd, 0x0a, 0x75, 0x0c, 0x21, 0xa8, 0x69, 0xe2, 0x1b, 0xb0, + 0x38, 0x18, 0x1e, 0xd8, 0x96, 0x7f, 0xa4, 0x05, 0x56, 0x9f, 0xb0, 0x79, 0x5c, 0x50, 0xcb, 0x02, + 0x46, 0xed, 0x6c, 0x64, 0x66, 0x16, 0xce, 0x3a, 0x33, 0xf1, 0x9b, 0xb0, 0xe4, 0x91, 0x81, 0x6d, + 0x19, 0x7a, 0x40, 0x4c, 0xed, 0xd0, 0x73, 0xfb, 0xf2, 0xdc, 0x9a, 0xb4, 0x5e, 0x52, 0xab, 0x31, + 0xf8, 0x89, 0xe7, 0xf6, 0xd9, 0x48, 0x42, 0xab, 0xd2, 0xa8, 0x0c, 0xe7, 0x19, 0xda, 0x62, 0x04, + 0xdc, 0x26, 0xa7, 0x94, 0xd1, 0xa8, 0x9b, 0x16, 0xb8, 0xf2, 0xc2, 0x5a, 0x7e, 0xbd, 0xa4, 0x96, + 0x23, 0x58, 0xd7, 0xc5, 0x0d, 0x28, 0x1b, 0x6e, 0x7f, 0xe0, 0x11, 0xdf, 0xa7, 0x46, 0x5b, 0x5c, + 0x93, 0xd6, 0xab, 0x1b, 0xaf, 0xa5, 0x39, 0xad, 0xc7, 0x08, 0xd4, 0x6b, 0x3c, 0x2e, 0xb4, 0xda, + 0xad, 0x86, 0x9a, 0xec, 0x87, 0xef, 0xc2, 0xf9, 0xa1, 0x13, 0x02, 0x88, 0xc9, 0x0d, 0xb4, 0xb4, + 0x26, 0xad, 0x57, 0x1e, 0x4b, 0xf7, 0x54, 0x94, 0x6c, 0xa3, 0xa6, 0x8a, 0x1f, 0xc2, 0x05, 0x67, + 0xd8, 0xd7, 0xfa, 0x5c, 0x3f, 0xbe, 0x66, 0x39, 0x1a, 0xb3, 0x63, 0xb9, 0xcc, 0x66, 0x84, 0x74, + 0x5f, 0xc5, 0xce, 0xb0, 0x2f, 0xd4, 0xe7, 0x37, 0x9d, 0x4d, 0xda, 0x88, 0xd7, 0x00, 0xc8, 0x09, + 0x71, 0x02, 0x2e, 0xf6, 0xc5, 0x35, 0x69, 0xbd, 0x40, 0xc9, 0x97, 0x18, 0x90, 0xc9, 0xbd, 0x01, + 0x4b, 0x24, 0x32, 0x31, 0x2a, 0x17, 0x5f, 0xae, 0x30, 0xe1, 0x5f, 0x4b, 0x0f, 0x29, 0x6d, 0x87, + 0x6a, 0x95, 0xa4, 0xed, 0xf2, 0xcd, 0x14, 0x19, 0xdd, 0xee, 0xb9, 0x72, 0x95, 0xab, 0x21, 0x06, + 0xd7, 0xec, 0x9e, 0x8b, 0xdf, 0x02, 0x94, 0x40, 0x1c, 0xe8, 0x9e, 0xde, 0x97, 0x97, 0xd6, 0xa4, + 0xf5, 0x45, 0x35, 0x41, 0x60, 0x8f, 0x82, 0xf1, 0x2d, 0xa8, 0x0a, 0xe7, 0x7f, 0x42, 0x3c, 0x26, + 0x6c, 0xc4, 0x10, 0x2b, 0x1c, 0xfa, 0x9c, 0x03, 0xf1, 0xc7, 0x70, 0x39, 0xa5, 0x58, 0xed, 0xe0, + 0xd1, 0x43, 0x8d, 0x38, 0x86, 0x6b, 0x12, 0x53, 0x3e, 0xbf, 0x26, 0xad, 0x17, 0x1f, 0xcf, 0x1d, + 0xea, 0xb6, 0x4f, 0xd4, 0x8b, 0x49, 0x5d, 0x6f, 0x3e, 0x7a, 0xd8, 0xe0, 0x48, 0x54, 0xeb, 0xae, + 0x67, 0x12, 0xea, 0xcc, 0x99, 0x65, 0x60, 0xf6, 0x99, 0x72, 0x08, 0xa3, 0x86, 0xf1, 0x06, 0x2c, + 0x99, 0xc4, 0xb6, 0x4e, 0x88, 0xa7, 0xe9, 0x42, 0x9a, 0xcb, 0x6b, 0xd2, 0x7a, 0x5e, 0xad, 0x08, + 0x70, 0x8d, 0x8b, 0xf3, 0x3a, 0x94, 0xfb, 0xba, 0x77, 0x4c, 0x3c, 0x8d, 0x85, 0xa5, 0x15, 0xe6, + 0x71, 0x80, 0x83, 0x58, 0x00, 0x59, 0x07, 0x14, 0xbc, 0x74, 0x2c, 0x53, 0xb3, 0x89, 0xee, 0x07, + 0xda, 0x81, 0x15, 0xf8, 0xf2, 0x45, 0xaa, 0x17, 0xb5, 0xca, 0xe0, 0x3b, 0x14, 0xbc, 0x69, 0x05, + 0x3e, 0xfd, 0x24, 0xc7, 0xec, 0xbb, 0x21, 0xe2, 0x25, 0x86, 0x58, 0x61, 0xe0, 0x5d, 0x57, 0xe0, + 0xdd, 0x87, 0xe5, 0x23, 0xab, 0x77, 0x44, 0xfc, 0x40, 0x4b, 0xce, 0x42, 0x39, 0x54, 0xf6, 0x79, + 0xd1, 0xda, 0x89, 0xe7, 0xe3, 0xeb, 0x00, 0xce, 0xd0, 0xb6, 0x35, 0xee, 0x38, 0x2e, 0x27, 0x65, + 0x54, 0xa2, 0x0d, 0xdc, 0xb3, 0x61, 0x28, 0x0c, 0x87, 0x96, 0x29, 0x5f, 0x61, 0x8a, 0x64, 0xcf, + 0xf8, 0x0e, 0x2c, 0x53, 0x33, 0x34, 0x8e, 0x86, 0xce, 0xb1, 0xcf, 0xa6, 0x9b, 0xd6, 0xf7, 0x7b, + 0xf2, 0x55, 0x36, 0x4e, 0xe4, 0x0c, 0xfb, 0x75, 0xd6, 0x42, 0x67, 0xdc, 0xae, 0xdf, 0xc3, 0xdf, + 0x82, 0x95, 0xc0, 0x0d, 0x74, 0x9b, 0x77, 0xa0, 0xa8, 0xdc, 0xd0, 0xaf, 0x31, 0xfc, 0xf3, 0xac, + 0x8d, 0xf5, 0xd8, 0xf5, 0x7b, 0xcc, 0xcc, 0x2f, 0x43, 0x91, 0xa3, 0x5a, 0xa6, 0xfc, 0x1a, 0x43, + 0x5a, 0x60, 0xef, 0x4d, 0x13, 0x3f, 0x00, 0xcc, 0x98, 0x4e, 0xcf, 0xe2, 0xd5, 0x24, 0xf3, 0x88, + 0x22, 0xec, 0x25, 0x94, 0xac, 0xfc, 0x6d, 0x1e, 0x2e, 0x74, 0x2c, 0xa7, 0x67, 0x93, 0x51, 0xcf, + 0x96, 0x76, 0x38, 0xd2, 0x99, 0x1d, 0xce, 0x98, 0x1f, 0xc9, 0x65, 0xfb, 0x91, 0x81, 0x7e, 0x6a, + 0xbb, 0xba, 0x98, 0xd8, 0x79, 0xe6, 0xd3, 0xcb, 0x02, 0xc6, 0x46, 0x7a, 0x1b, 0x2a, 0x74, 0x8a, + 0xeb, 0x06, 0xf5, 0x5b, 0xee, 0x30, 0x60, 0xa1, 0x2d, 0x1a, 0xc9, 0x62, 0xd4, 0xd6, 0x1e, 0x06, + 0x23, 0xd3, 0x78, 0x2e, 0x63, 0x1a, 0x4f, 0x9d, 0x04, 0xf3, 0x5f, 0x64, 0x12, 0x2c, 0x8c, 0x4f, + 0x82, 0x11, 0x3f, 0x5f, 0x64, 0xd6, 0x98, 0xf4, 0xf3, 0x69, 0xbb, 0x2a, 0x4d, 0xb0, 0xab, 0x6c, + 0x45, 0xc2, 0x74, 0x45, 0x3e, 0x87, 0xe5, 0x4d, 0xcf, 0x3d, 0x26, 0x5e, 0x83, 0xa6, 0x0e, 0x91, + 0x16, 0xdf, 0x02, 0x74, 0xc0, 0xc0, 0x4c, 0x34, 0x2c, 0xfd, 0x91, 0x25, 0xc6, 0xd7, 0x12, 0x87, + 0x47, 0x59, 0x11, 0x0d, 0x94, 0x3c, 0x87, 0xc8, 0xb1, 0x76, 0xfe, 0xa2, 0xfc, 0x7b, 0x1e, 0xaa, + 0x75, 0xb7, 0xdf, 0xd7, 0x1d, 0xb3, 0xee, 0x3a, 0x0e, 0x31, 0x02, 0xea, 0x77, 0x0c, 0xdb, 0xa2, + 0xe2, 0x0e, 0xfd, 0x0e, 0x0f, 0x7a, 0x15, 0x0e, 0x0d, 0xfd, 0xce, 0x07, 0x50, 0xd6, 0x87, 0xc1, + 0x91, 0xd6, 0x27, 0xc1, 0x91, 0x6b, 0x32, 0xaa, 0xd5, 0x0d, 0x39, 0x6d, 0x41, 0xb5, 0x61, 0x70, + 0xb4, 0xcb, 0xda, 0x55, 0xd0, 0xa3, 0x67, 0xea, 0x04, 0x12, 0x5d, 0x79, 0x60, 0x15, 0x51, 0x2b, + 0xc6, 0x62, 0xa1, 0xf5, 0x2a, 0x94, 0x18, 0xa6, 0x08, 0xe4, 0x54, 0x25, 0x45, 0x0a, 0x60, 0x39, + 0xd7, 0x3b, 0x80, 0xd8, 0x67, 0x0c, 0xd7, 0x8e, 0x58, 0xe5, 0x09, 0x92, 0x74, 0x4f, 0x5d, 0x0a, + 0x9b, 0x42, 0x7e, 0xef, 0xc0, 0xf2, 0xc0, 0x73, 0x5f, 0x9e, 0x6a, 0x81, 0xab, 0x09, 0x99, 0x0d, + 0x3d, 0x5b, 0x84, 0x41, 0xc4, 0x9a, 0xba, 0x2e, 0x97, 0xf1, 0xbe, 0x67, 0xe3, 0x3b, 0x80, 0x5d, + 0xcf, 0xea, 0x59, 0x8e, 0x6e, 0x6b, 0x03, 0xcf, 0x72, 0x0c, 0x6b, 0xa0, 0xdb, 0xcc, 0x2a, 0x4a, + 0xea, 0xf9, 0xb0, 0x65, 0x2f, 0x6c, 0xc0, 0xef, 0x24, 0xd0, 0x63, 0x8e, 0x8b, 0x9c, 0x78, 0xd8, + 0x52, 0x0b, 0x39, 0xbf, 0x07, 0x2b, 0x69, 0x6c, 0x21, 0xc4, 0x12, 0xc3, 0xc7, 0x49, 0x7c, 0x21, + 0xb2, 0xef, 0x40, 0xe5, 0x90, 0xe8, 0xc1, 0xd0, 0x23, 0xda, 0xa1, 0xad, 0xf7, 0x7c, 0x66, 0x2f, + 0xe5, 0x8d, 0x2b, 0x69, 0x79, 0x3f, 0xe1, 0x28, 0x4f, 0x28, 0x86, 0xba, 0x78, 0x98, 0x78, 0x53, + 0x7e, 0x5f, 0x82, 0xc5, 0x64, 0x33, 0xfe, 0x00, 0x2e, 0xf8, 0xc3, 0xc1, 0xc0, 0xf5, 0x02, 0x9f, + 0xf3, 0xe0, 0x91, 0x43, 0x8f, 0xf8, 0x47, 0xcc, 0x7e, 0x22, 0x4b, 0x5c, 0x0e, 0x71, 0x28, 0x2f, + 0x2a, 0xc7, 0xc0, 0xdf, 0x85, 0xd5, 0xa8, 0xab, 0x10, 0x25, 0xcb, 0x68, 0xb5, 0x28, 0xe7, 0xca, + 0x25, 0x69, 0x5c, 0x0d, 0x91, 0x33, 0x2c, 0x58, 0xf9, 0x3d, 0x09, 0x50, 0xda, 0x00, 0x89, 0xc9, + 0x42, 0x1f, 0xf1, 0x68, 0xb4, 0x19, 0x31, 0x41, 0x0e, 0x0d, 0x55, 0x9a, 0x65, 0x00, 0xb9, 0x89, + 0x06, 0xb0, 0x0e, 0xa8, 0xaf, 0xbf, 0x0c, 0x53, 0x88, 0xd0, 0x31, 0x51, 0x1f, 0x5b, 0xed, 0xeb, + 0x2f, 0x85, 0x7f, 0x64, 0x79, 0xf1, 0x1f, 0x48, 0xb0, 0x2c, 0x78, 0xe2, 0xc3, 0xf6, 0x07, 0xae, + 0xe3, 0x93, 0xcc, 0x99, 0x21, 0x8d, 0xcf, 0x8c, 0x0d, 0x28, 0x7a, 0xa2, 0x0b, 0x63, 0x67, 0xcc, + 0xb1, 0x86, 0x76, 0xa0, 0x46, 0x78, 0x99, 0x43, 0xc9, 0x4f, 0x1a, 0x8a, 0xf2, 0x47, 0x12, 0xac, + 0x24, 0x18, 0xac, 0x1f, 0xe9, 0xb6, 0x4d, 0x68, 0x66, 0x9d, 0x25, 0x38, 0x69, 0x5c, 0x70, 0x0f, + 0xa1, 0x64, 0x84, 0x7d, 0x66, 0xb0, 0x18, 0x23, 0xbe, 0x22, 0x8f, 0xdf, 0x83, 0x62, 0x64, 0xef, + 0x59, 0x13, 0x5e, 0x9a, 0x3d, 0xe1, 0x73, 0xe9, 0x09, 0xaf, 0xfc, 0xbd, 0x04, 0x95, 0x6d, 0x72, + 0xda, 0x39, 0xd2, 0x3d, 0x62, 0x52, 0x0b, 0xc2, 0x35, 0xa8, 0x1c, 0x47, 0x00, 0xd7, 0xe4, 0xf9, + 0x79, 0x75, 0xe3, 0xea, 0x58, 0x20, 0x8b, 0x51, 0xd4, 0x74, 0x0f, 0x1a, 0x08, 0x8f, 0x74, 0xff, + 0x88, 0xad, 0x4c, 0xfc, 0xec, 0xc5, 0x42, 0xb8, 0x70, 0x51, 0x13, 0x98, 0xf8, 0x3b, 0x70, 0x49, + 0xb7, 0x6d, 0xf7, 0x45, 0x7b, 0x18, 0xb4, 0x0f, 0xdb, 0x34, 0x4c, 0x6c, 0xf1, 0x54, 0xe8, 0x34, + 0x1d, 0xca, 0x26, 0x61, 0x29, 0xbf, 0x58, 0x88, 0x2c, 0xbf, 0x33, 0x3c, 0xf0, 0x0d, 0xcf, 0x3a, + 0x60, 0x4b, 0xa3, 0xc0, 0x1d, 0x58, 0x86, 0x30, 0x78, 0xfe, 0x82, 0x15, 0x58, 0xf4, 0x39, 0x0a, + 0xcb, 0x0f, 0xc5, 0x8a, 0x2c, 0x05, 0xc3, 0x1f, 0xc3, 0x82, 0x3f, 0x3c, 0xa0, 0x49, 0x16, 0x0b, + 0xb7, 0xd5, 0x8d, 0x37, 0xc6, 0x92, 0xf2, 0xd4, 0xa7, 0xee, 0x76, 0x38, 0xb6, 0x1a, 0x76, 0xa3, + 0xf1, 0xcd, 0x70, 0x1d, 0x7f, 0xd8, 0x27, 0x1e, 0x8d, 0x6f, 0x05, 0xbe, 0x8e, 0x09, 0x41, 0x4d, + 0x93, 0x2e, 0x27, 0x3d, 0x1a, 0xed, 0xfc, 0x80, 0xb6, 0xcf, 0xb1, 0xf6, 0x92, 0x80, 0x34, 0x4d, + 0x9a, 0x1a, 0x44, 0xfd, 0x99, 0x8a, 0xc5, 0x12, 0x23, 0x04, 0x32, 0x05, 0xdf, 0x82, 0xea, 0xc0, + 0xb3, 0x5c, 0xcf, 0x0a, 0x4e, 0x35, 0x9b, 0x9c, 0x10, 0xee, 0x53, 0xe7, 0xd4, 0x4a, 0x08, 0xdd, + 0xa1, 0x40, 0xbc, 0x0a, 0x0b, 0xe6, 0xd0, 0xd3, 0x0f, 0x6c, 0xc2, 0x9c, 0x68, 0xf1, 0x71, 0x21, + 0xf0, 0x86, 0x44, 0x0d, 0x81, 0xb8, 0x01, 0x88, 0xad, 0x1a, 0xa3, 0xe9, 0x6c, 0x71, 0xef, 0x59, + 0x1e, 0xd5, 0x7d, 0x6a, 0x99, 0xae, 0x56, 0x59, 0xa7, 0x08, 0x96, 0x5a, 0x27, 0xc2, 0xd9, 0xd6, + 0x89, 0x74, 0x04, 0x1e, 0xd1, 0x4d, 0x2d, 0x4a, 0x51, 0xd8, 0x1a, 0xa4, 0xa8, 0x56, 0x28, 0xb4, + 0x1e, 0x02, 0xf1, 0x3b, 0x30, 0xcf, 0x13, 0x75, 0xb6, 0xee, 0x28, 0x6f, 0xac, 0x64, 0x15, 0x67, + 0x54, 0x81, 0x83, 0x7f, 0x08, 0x4b, 0x96, 0x63, 0x05, 0x96, 0x6e, 0xef, 0xb9, 0x3e, 0xaf, 0x07, + 0x54, 0x58, 0x44, 0xbd, 0x3b, 0x43, 0x8b, 0xcd, 0x74, 0xaf, 0xc7, 0xf3, 0x3b, 0x7a, 0x40, 0xfc, + 0x40, 0x1d, 0x25, 0x87, 0x3f, 0x86, 0x6b, 0xf1, 0xda, 0x2e, 0x69, 0x39, 0x9a, 0x1f, 0xe8, 0x01, + 0x61, 0xeb, 0x95, 0xa2, 0x7a, 0x25, 0xc2, 0xe9, 0x24, 0x50, 0x3a, 0x14, 0x03, 0x3f, 0x82, 0x95, + 0x43, 0xd7, 0x33, 0xe8, 0xca, 0x70, 0x60, 0x19, 0x9a, 0xe1, 0x11, 0x9d, 0x31, 0xba, 0x94, 0x50, + 0x10, 0x66, 0x18, 0x5d, 0x8a, 0x50, 0x17, 0xed, 0xb8, 0x0d, 0x37, 0xd3, 0xba, 0xf2, 0x5c, 0xdb, + 0x3e, 0xd0, 0x8d, 0x63, 0x8d, 0x6a, 0x93, 0xb3, 0x40, 0x0c, 0xb6, 0xba, 0x61, 0x79, 0xdd, 0xf5, + 0xa4, 0x92, 0x54, 0x81, 0xbb, 0x25, 0x50, 0x3b, 0xc4, 0x48, 0xcf, 0x7a, 0x12, 0xe8, 0x6c, 0x99, + 0x53, 0x9e, 0x3c, 0xeb, 0x49, 0xa0, 0xab, 0xe9, 0x1e, 0xca, 0x26, 0x2c, 0x08, 0xfb, 0xc7, 0x15, + 0x28, 0x35, 0x5e, 0x1a, 0xf6, 0xd0, 0xb7, 0x4e, 0xc2, 0xc2, 0x16, 0xc3, 0x43, 0x12, 0x5e, 0x84, + 0xe2, 0x13, 0xdd, 0xb2, 0xdd, 0x13, 0xe2, 0xa1, 0x1c, 0xae, 0x02, 0x6c, 0x93, 0x53, 0x4d, 0xb4, + 0xe6, 0x95, 0xb7, 0x61, 0x69, 0x44, 0xfa, 0xb4, 0x33, 0x97, 0x3f, 0x3a, 0x47, 0x3b, 0x37, 0x74, + 0xcf, 0xb6, 0xe8, 0x9b, 0xa4, 0xfc, 0x9b, 0x04, 0xd7, 0x85, 0xf2, 0xa2, 0xc4, 0x8e, 0x98, 0x4c, + 0x50, 0x51, 0x36, 0x97, 0x3d, 0xf9, 0xd3, 0xb3, 0x2e, 0x37, 0x3a, 0xeb, 0xb2, 0x13, 0x95, 0xfc, + 0xab, 0x25, 0x2a, 0x85, 0x57, 0x4c, 0x54, 0xe6, 0x26, 0x25, 0x2a, 0xca, 0x9f, 0xe7, 0xe0, 0xcd, + 0x19, 0xe3, 0x8c, 0xe2, 0xe9, 0x2a, 0x40, 0x94, 0x04, 0xfb, 0x2c, 0x20, 0x54, 0xd4, 0x04, 0x64, + 0xd6, 0xc8, 0xbf, 0x9f, 0x88, 0xb3, 0x79, 0x36, 0x59, 0x3e, 0xce, 0x9c, 0x2c, 0xb3, 0xf8, 0xb8, + 0xbb, 0xe3, 0xba, 0xc7, 0xc3, 0x01, 0x73, 0x86, 0x71, 0x44, 0xfe, 0x16, 0xcc, 0x11, 0xcf, 0x73, + 0x3d, 0x26, 0x9b, 0xf1, 0xda, 0x2a, 0x8b, 0xa7, 0x0d, 0x8a, 0xa0, 0x72, 0x3c, 0x2c, 0xc3, 0x82, + 0x30, 0x70, 0x21, 0x9e, 0xf0, 0x55, 0xb9, 0x05, 0x10, 0x7f, 0x02, 0x97, 0xa9, 0xe9, 0x19, 0x06, + 0xf1, 0x7d, 0x6e, 0x6d, 0xd4, 0xc2, 0xa8, 0xb5, 0x29, 0x7f, 0x95, 0x03, 0x2c, 0x58, 0x16, 0xe8, + 0x4c, 0xff, 0x5f, 0xc8, 0x2a, 0xde, 0x86, 0x0a, 0xd5, 0x17, 0xf5, 0xa8, 0xac, 0x68, 0xc9, 0x04, + 0x14, 0xc5, 0xa4, 0x74, 0xdb, 0x04, 0x13, 0x2a, 0xbc, 0x9a, 0x09, 0xcd, 0xbd, 0xa2, 0x09, 0xcd, + 0x4f, 0xcc, 0x75, 0xdf, 0x07, 0x59, 0x37, 0x4f, 0xe8, 0x7a, 0xd3, 0x27, 0xa6, 0x66, 0x5b, 0x7e, + 0x40, 0x9c, 0x30, 0xa4, 0xf0, 0x04, 0xfc, 0x62, 0xdc, 0xbe, 0x23, 0x9a, 0x69, 0x70, 0x51, 0x7e, + 0x9a, 0x87, 0x2b, 0xe3, 0x12, 0x8c, 0xec, 0xed, 0x76, 0xb8, 0x5a, 0xa2, 0xda, 0xb3, 0x0c, 0xb2, + 0xef, 0xd9, 0x22, 0x0d, 0x19, 0x83, 0xe3, 0x7b, 0xb0, 0x3c, 0x0a, 0xeb, 0xda, 0xbe, 0x58, 0xed, + 0x66, 0x35, 0xe1, 0xf6, 0x98, 0x39, 0x3e, 0xc8, 0x34, 0xc7, 0x0c, 0xce, 0xb2, 0x2d, 0x30, 0xad, + 0xe2, 0xc2, 0x4c, 0x15, 0xcf, 0x4d, 0x51, 0x71, 0x64, 0xcd, 0xf3, 0xaf, 0x6e, 0xcd, 0x0b, 0x29, + 0x6b, 0x66, 0x6b, 0x6d, 0xbe, 0x90, 0x3a, 0xf2, 0xdc, 0x61, 0xef, 0x48, 0xf3, 0xb9, 0x18, 0xd8, + 0x72, 0xaa, 0x98, 0x5e, 0x6b, 0xb3, 0x55, 0x15, 0x47, 0x8b, 0x85, 0xa5, 0x3c, 0x48, 0xcd, 0x87, + 0x45, 0x28, 0xaa, 0xc4, 0xb4, 0x3c, 0x62, 0x50, 0xaf, 0x59, 0x86, 0x05, 0xb1, 0x0e, 0x40, 0x52, + 0x62, 0x76, 0xe4, 0x94, 0xff, 0xca, 0xc3, 0x52, 0x38, 0xa1, 0x45, 0xf5, 0x75, 0xc2, 0xd4, 0xb8, + 0x0e, 0xe5, 0xa8, 0x68, 0x1b, 0xd7, 0x63, 0x43, 0xd0, 0x58, 0x1e, 0x93, 0xcf, 0xc8, 0x63, 0xd2, + 0x45, 0xdf, 0x82, 0x28, 0x71, 0x24, 0x8b, 0xbe, 0x37, 0xa1, 0x24, 0x0a, 0x76, 0xc4, 0x4c, 0x4b, + 0x3e, 0x86, 0xa7, 0xd2, 0x8b, 0xf9, 0x33, 0xa6, 0x17, 0x71, 0xde, 0xb0, 0x70, 0x86, 0xbc, 0xe1, + 0x12, 0xcc, 0x91, 0x81, 0x6b, 0x1c, 0xf1, 0x6a, 0x04, 0x8d, 0x9e, 0xfc, 0x1d, 0xd7, 0xe1, 0xea, + 0xd0, 0x27, 0x9e, 0x36, 0xf0, 0xdc, 0x13, 0xcb, 0x24, 0xa6, 0x96, 0x1e, 0x52, 0x29, 0x11, 0xb3, + 0x65, 0x8a, 0xb8, 0x27, 0xf0, 0xf6, 0x92, 0x83, 0xfc, 0x0c, 0x56, 0xa2, 0x6e, 0x3a, 0x73, 0x59, + 0x5a, 0x9f, 0x66, 0xd9, 0xc0, 0x8c, 0x68, 0x2d, 0xcd, 0x59, 0xd8, 0xb3, 0xc6, 0x10, 0x69, 0x6e, + 0xfd, 0x58, 0x44, 0x52, 0x15, 0x0f, 0xc6, 0xda, 0xa8, 0x96, 0x78, 0x1e, 0xc1, 0xf9, 0x2f, 0xf3, + 0x6a, 0x0a, 0x03, 0x35, 0x28, 0x44, 0xf9, 0xe3, 0x1c, 0x94, 0xc3, 0x74, 0x87, 0x38, 0xe6, 0xa8, + 0x5a, 0xa5, 0x31, 0xb5, 0xce, 0xac, 0xc3, 0xbf, 0x0e, 0x8b, 0xc9, 0x22, 0x72, 0xb8, 0x78, 0xb9, + 0xaf, 0x96, 0x13, 0xb5, 0x63, 0xfc, 0x76, 0x46, 0x89, 0xb2, 0x10, 0x4a, 0x77, 0xb4, 0x4a, 0xf9, + 0xd6, 0x78, 0x95, 0x32, 0xaa, 0x4f, 0x9d, 0xad, 0x50, 0x39, 0x3f, 0xa5, 0x50, 0xb9, 0x06, 0x45, + 0xcb, 0xe7, 0xc5, 0x43, 0x66, 0x0d, 0x91, 0x8d, 0x2d, 0x58, 0x3e, 0xab, 0x1b, 0x2a, 0x7f, 0x23, + 0x45, 0x31, 0x83, 0x0a, 0x49, 0x25, 0x06, 0xb1, 0x06, 0xc1, 0x2f, 0x41, 0x56, 0x8f, 0x01, 0x12, + 0xa9, 0x75, 0x7e, 0x76, 0x6a, 0x5d, 0xea, 0x47, 0x59, 0xf5, 0x84, 0x91, 0x16, 0x26, 0x8f, 0x54, + 0xf9, 0xc3, 0xb8, 0x0c, 0x40, 0xc7, 0xc1, 0x5c, 0xd1, 0x2f, 0x61, 0x14, 0x91, 0xdb, 0xcb, 0x67, + 0x6e, 0x90, 0x4e, 0x75, 0x7b, 0x05, 0xe6, 0x53, 0xa2, 0x20, 0xfe, 0x63, 0x29, 0xaa, 0x94, 0x89, + 0x81, 0x8f, 0x2e, 0x98, 0xa4, 0xb1, 0x05, 0x53, 0x5a, 0x88, 0x94, 0xbd, 0xb3, 0x0b, 0xf1, 0x1d, + 0x40, 0x1e, 0x11, 0xd5, 0xf5, 0x53, 0xcd, 0x70, 0x87, 0x4e, 0xc0, 0xd4, 0xc0, 0x36, 0x48, 0x96, + 0xe2, 0xa6, 0x3a, 0x6d, 0x49, 0x6e, 0x01, 0x16, 0x92, 0x5b, 0x80, 0xca, 0x2f, 0x0a, 0x00, 0x61, + 0xa9, 0xc0, 0x38, 0x9e, 0xcd, 0xf2, 0x87, 0x50, 0xa4, 0x84, 0x58, 0xf9, 0x3e, 0xc7, 0x84, 0xb6, + 0x96, 0x19, 0xc5, 0x6a, 0xc6, 0xf1, 0xdd, 0x9a, 0x71, 0xcc, 0x57, 0x90, 0x3a, 0x7f, 0x18, 0x33, + 0x9a, 0xfc, 0x2b, 0x8c, 0xb7, 0x03, 0xe8, 0x44, 0xb7, 0x2d, 0x93, 0x2f, 0x08, 0x92, 0xa9, 0xd7, + 0xfa, 0x44, 0x06, 0x9e, 0x47, 0x1d, 0xb8, 0x12, 0x97, 0x4e, 0xd2, 0x00, 0xca, 0xd0, 0xd8, 0x86, + 0xf7, 0x95, 0x31, 0x17, 0x1c, 0xed, 0x66, 0xa6, 0x2a, 0xdd, 0x59, 0x7e, 0x60, 0xfe, 0x15, 0xfc, + 0xc0, 0xc2, 0x04, 0x3f, 0x90, 0x8e, 0x3e, 0xbc, 0x8a, 0x1c, 0x47, 0x1f, 0xe5, 0x2d, 0x58, 0x10, + 0x72, 0xa5, 0x0b, 0x8e, 0xa6, 0x63, 0x5a, 0x27, 0x96, 0x39, 0xd4, 0x6d, 0x74, 0x8e, 0xbe, 0xd7, + 0x87, 0xfd, 0xa1, 0xcd, 0xb7, 0xa1, 0x25, 0xe5, 0x77, 0x25, 0x58, 0x1a, 0x11, 0x01, 0x5e, 0x85, + 0x2b, 0xfb, 0x23, 0x9b, 0x67, 0x75, 0xd7, 0xf3, 0x86, 0x6c, 0x1d, 0x87, 0xce, 0xe1, 0x8b, 0x80, + 0xb7, 0x48, 0x62, 0x27, 0x8e, 0xf5, 0x42, 0x12, 0x5e, 0x01, 0x54, 0x3f, 0x22, 0xc6, 0xb1, 0x3f, + 0xec, 0xef, 0x5a, 0x7e, 0x5f, 0x0f, 0x8c, 0x23, 0x94, 0xc3, 0x97, 0xe1, 0x02, 0xdb, 0x49, 0xdb, + 0x22, 0x1d, 0xe2, 0x59, 0xba, 0x6d, 0x7d, 0x4e, 0x78, 0x87, 0x3c, 0x5e, 0x86, 0xa5, 0x2d, 0x12, + 0xee, 0x58, 0x71, 0x60, 0x41, 0xf9, 0x9f, 0xd8, 0x1d, 0xd5, 0x8c, 0xe3, 0x28, 0xf1, 0x9a, 0x69, + 0x75, 0x59, 0xb2, 0xce, 0xbd, 0x82, 0xac, 0xf3, 0x13, 0x64, 0xfd, 0xcb, 0x4b, 0xe2, 0x47, 0xd4, + 0x36, 0x3f, 0xaa, 0xb6, 0x03, 0xb8, 0x1a, 0x0d, 0x9c, 0xaa, 0xa7, 0x2e, 0x06, 0x57, 0x3f, 0x62, + 0x5b, 0xde, 0x33, 0x25, 0xa0, 0x40, 0xc9, 0xf2, 0x35, 0x9d, 0xf5, 0x4d, 0x97, 0x4f, 0x8b, 0x96, + 0xcf, 0x49, 0x2a, 0xcf, 0xa3, 0x80, 0xf8, 0xc4, 0x76, 0x5f, 0xcc, 0xa6, 0xf9, 0x06, 0x54, 0x05, + 0xf7, 0x7b, 0xc4, 0xeb, 0x73, 0x99, 0xe6, 0xd6, 0x2b, 0xea, 0x08, 0x54, 0xe9, 0x46, 0x4a, 0xdb, + 0x77, 0xfc, 0xa8, 0x14, 0x35, 0x93, 0xfc, 0xf4, 0x25, 0x88, 0xf2, 0x17, 0x52, 0x22, 0x7e, 0x93, + 0xe3, 0x2f, 0x4b, 0xef, 0x4b, 0x45, 0xa4, 0x7b, 0xb0, 0x12, 0xf6, 0x4d, 0xed, 0xc4, 0xb3, 0x90, + 0xa4, 0xe2, 0x50, 0x1e, 0xf1, 0x86, 0xbc, 0xf2, 0x21, 0xc8, 0x82, 0x79, 0x95, 0xe8, 0xc6, 0x11, + 0x31, 0x1b, 0x8e, 0xd9, 0x3e, 0xec, 0x86, 0x09, 0xe6, 0xd4, 0x91, 0x28, 0xcf, 0xa3, 0xf2, 0x6c, + 0xdd, 0x76, 0x7d, 0x12, 0xe5, 0xab, 0x33, 0x03, 0xda, 0x0c, 0x91, 0x8e, 0xd0, 0x0d, 0x6d, 0xec, + 0x4b, 0xab, 0xea, 0xb7, 0x25, 0x78, 0x23, 0x1a, 0xad, 0x08, 0x2c, 0xfb, 0x8e, 0x6e, 0x1c, 0x3b, + 0xee, 0x0b, 0x76, 0x8c, 0xc5, 0x8c, 0xb2, 0xa3, 0x99, 0x9f, 0xfa, 0x08, 0xca, 0xb1, 0x9a, 0xa8, + 0xc5, 0xcd, 0x0c, 0x02, 0x10, 0xe9, 0xc9, 0x57, 0x7e, 0x10, 0x05, 0x59, 0xb1, 0x46, 0x1e, 0x61, + 0x5d, 0x1a, 0xb5, 0x8a, 0x38, 0x5d, 0xce, 0xcd, 0x4e, 0x97, 0x95, 0xff, 0x95, 0xe0, 0xe2, 0xc8, + 0x22, 0xe2, 0x8c, 0xdf, 0x19, 0x5b, 0x14, 0xe4, 0x32, 0x4e, 0x82, 0xbc, 0x03, 0xc8, 0xd6, 0x47, + 0xb2, 0x1e, 0x6a, 0xa8, 0x79, 0x76, 0x64, 0xa7, 0x4a, 0xdb, 0x12, 0xd9, 0xdd, 0xf8, 0x06, 0x7f, + 0x21, 0x6b, 0x83, 0x7f, 0x24, 0x51, 0x9e, 0x1b, 0x4d, 0x94, 0xf1, 0xdb, 0x50, 0x8d, 0x58, 0xf3, + 0x88, 0x6e, 0x9e, 0x8a, 0x1d, 0x4f, 0x9e, 0xdd, 0x47, 0x6c, 0xab, 0xb4, 0x49, 0x79, 0x09, 0x8b, + 0x42, 0x00, 0x3c, 0x5e, 0xcc, 0x18, 0x76, 0xe4, 0x40, 0x73, 0xaf, 0x9e, 0x40, 0xe5, 0xd3, 0x09, + 0x54, 0x25, 0x72, 0x07, 0x7b, 0x96, 0xd3, 0x4b, 0xbe, 0xba, 0x4e, 0x2f, 0x69, 0xda, 0xc2, 0x96, + 0x3a, 0x81, 0x1e, 0xcc, 0x54, 0xcb, 0xac, 0x9a, 0xb5, 0xf2, 0xdf, 0x05, 0xb8, 0x96, 0x45, 0x58, + 0xcd, 0x5e, 0x65, 0x8f, 0x7d, 0xe0, 0x7d, 0x00, 0x36, 0x30, 0xcd, 0xa0, 0x0b, 0x9f, 0xdc, 0xac, + 0x30, 0x52, 0x62, 0xc8, 0x75, 0xba, 0xc0, 0xb9, 0x09, 0x15, 0xde, 0x33, 0x96, 0x07, 0x5b, 0x46, + 0x32, 0x60, 0x98, 0x42, 0xae, 0x02, 0xf4, 0xfd, 0x9e, 0xaa, 0x07, 0xa4, 0x2d, 0xf6, 0xc0, 0x25, + 0x35, 0x01, 0xc1, 0xb7, 0x01, 0xf5, 0xfd, 0x9e, 0x58, 0x42, 0x0f, 0x86, 0x01, 0xc5, 0x9a, 0x63, + 0x58, 0x63, 0x70, 0x81, 0x4b, 0x7b, 0x46, 0x93, 0x98, 0x59, 0x02, 0xc7, 0x4d, 0xc1, 0xb1, 0x02, + 0xa9, 0xb2, 0xbc, 0x58, 0xe3, 0xa7, 0x4b, 0xf5, 0xb7, 0x01, 0xe9, 0x27, 0xba, 0x65, 0xeb, 0x07, + 0x76, 0x14, 0x40, 0x78, 0xba, 0x32, 0x06, 0xc7, 0xeb, 0xb0, 0x34, 0xa4, 0x0e, 0x23, 0xf6, 0x14, + 0x6c, 0x89, 0x59, 0x50, 0x47, 0xc1, 0x78, 0x13, 0xae, 0x1d, 0xd8, 0x2e, 0x05, 0x85, 0xfa, 0x68, + 0x3b, 0xfb, 0x02, 0xc7, 0x17, 0x1b, 0x9b, 0x45, 0x75, 0x2a, 0x0e, 0x35, 0x32, 0xdd, 0x34, 0x69, + 0x06, 0xc3, 0xd6, 0x8d, 0x25, 0x35, 0x7c, 0xa5, 0x21, 0xcf, 0x08, 0xb7, 0x11, 0x3b, 0x96, 0x63, + 0xf0, 0x53, 0x3f, 0x25, 0x75, 0x04, 0x8a, 0xb1, 0x38, 0x38, 0x59, 0xe1, 0x87, 0x3b, 0xd8, 0xe9, + 0x48, 0x1a, 0x2e, 0xb9, 0x9c, 0x1a, 0x2f, 0x07, 0x96, 0x47, 0x4c, 0x56, 0x13, 0x97, 0xd4, 0x11, + 0xa8, 0xd0, 0xd9, 0xa6, 0x6e, 0x1c, 0xdb, 0x6e, 0x8f, 0x55, 0xbf, 0x0b, 0x6a, 0x02, 0xa2, 0x7c, + 0x0a, 0x97, 0x84, 0xc5, 0x3d, 0x25, 0xc1, 0x8e, 0xee, 0x27, 0xf6, 0x1b, 0xbe, 0xac, 0xa3, 0x4e, + 0x54, 0x91, 0x47, 0x69, 0x47, 0x06, 0x5d, 0x87, 0x25, 0xe6, 0x84, 0x12, 0xc1, 0x52, 0x9a, 0xbd, + 0xf2, 0xa8, 0xd8, 0x29, 0x46, 0x67, 0xc4, 0xe2, 0x5f, 0x87, 0xd7, 0xa2, 0x71, 0xf4, 0x75, 0xef, + 0x58, 0x33, 0x89, 0x4d, 0x02, 0xa2, 0x0d, 0xc2, 0xcd, 0x8b, 0x33, 0x84, 0xe7, 0x2b, 0x21, 0x85, + 0x5d, 0xdd, 0x3b, 0xde, 0x62, 0xfd, 0xc3, 0x3a, 0xba, 0xf2, 0x33, 0x29, 0x4a, 0xa7, 0x9e, 0x92, + 0x80, 0x45, 0x5d, 0xbf, 0x7d, 0x48, 0xad, 0xd2, 0x1f, 0xe8, 0xc6, 0xcc, 0x49, 0x7b, 0x0d, 0x4a, + 0x4e, 0x88, 0x2b, 0x1c, 0x75, 0x0c, 0xc0, 0x2d, 0x28, 0xb0, 0x2a, 0x46, 0x7e, 0xca, 0x06, 0x4b, + 0xd6, 0x57, 0xef, 0xb2, 0x9a, 0x06, 0xec, 0x35, 0xd4, 0x4e, 0xb3, 0xd3, 0x6d, 0xb4, 0xba, 0x2a, + 0xa3, 0xa3, 0x3c, 0x80, 0x02, 0xab, 0x68, 0x54, 0x21, 0xd1, 0x86, 0xce, 0x61, 0x0c, 0xd5, 0x56, + 0xbb, 0xa5, 0x25, 0x60, 0x12, 0x5e, 0x80, 0x7c, 0x6d, 0x67, 0x07, 0xe5, 0x94, 0xef, 0xc3, 0xcd, + 0x29, 0x9f, 0x3a, 0xab, 0x77, 0xba, 0x08, 0xf3, 0x2c, 0x10, 0xf0, 0x38, 0x5b, 0x52, 0xc5, 0x9b, + 0xe2, 0x44, 0xab, 0xe9, 0xa7, 0x24, 0x10, 0x07, 0x8c, 0x67, 0x90, 0x8a, 0x6a, 0x69, 0xb9, 0x64, + 0x2d, 0x6d, 0x3c, 0x46, 0xe5, 0x33, 0x62, 0x94, 0xf2, 0x9f, 0x52, 0x94, 0x2e, 0x45, 0x1f, 0xfc, + 0x9a, 0x78, 0xd8, 0x38, 0x41, 0x28, 0x9c, 0xa1, 0x9e, 0x36, 0x3e, 0xde, 0xb9, 0xac, 0xf1, 0xfe, + 0x46, 0x72, 0xb8, 0x6d, 0x8f, 0x6d, 0x75, 0x91, 0x2f, 0x23, 0xe7, 0x98, 0xcb, 0x3c, 0x9b, 0xab, + 0xd3, 0xd3, 0x98, 0xbf, 0x93, 0x60, 0x6d, 0xd2, 0xf7, 0xbf, 0x26, 0x62, 0x3f, 0x5b, 0x72, 0xa3, + 0xfc, 0x08, 0x2a, 0x62, 0x20, 0x2d, 0xf2, 0xa2, 0xfb, 0xd2, 0x99, 0xc5, 0x35, 0x5f, 0xfb, 0x69, + 0x41, 0x60, 0x6b, 0x3e, 0x31, 0x5c, 0xc7, 0x4c, 0xac, 0x13, 0xe9, 0xda, 0xaf, 0x1b, 0xd8, 0x1d, + 0x0e, 0xc7, 0x17, 0x61, 0x2e, 0x30, 0xc2, 0x0c, 0x8c, 0x21, 0x14, 0x02, 0xa3, 0x69, 0x2a, 0x3f, + 0x95, 0xe0, 0x42, 0xea, 0x9b, 0x67, 0x95, 0xd8, 0xd7, 0x7f, 0x91, 0xaa, 0xfc, 0x59, 0x3c, 0x0f, + 0x6b, 0x66, 0xbc, 0xf1, 0xd5, 0x75, 0xcf, 0x20, 0xda, 0xff, 0xaf, 0xe1, 0xa5, 0x77, 0xf9, 0x0a, + 0xcc, 0x4f, 0x25, 0x20, 0xca, 0xbf, 0xc6, 0xc6, 0x3c, 0xc6, 0xf3, 0x37, 0x48, 0x35, 0xcf, 0x60, + 0x31, 0xb9, 0xa5, 0xfe, 0xc5, 0x4f, 0x7a, 0x28, 0xff, 0x14, 0x07, 0xc7, 0x9a, 0x69, 0x26, 0x89, + 0x7e, 0xa5, 0x7a, 0xfe, 0x95, 0x11, 0xd6, 0x0b, 0x59, 0xd5, 0xb6, 0x24, 0xb7, 0x23, 0xc3, 0xfa, + 0x0f, 0x29, 0x0a, 0x89, 0x59, 0xc3, 0xfa, 0x06, 0x99, 0xc2, 0x5f, 0x4a, 0x91, 0xd7, 0x6b, 0x38, + 0xe6, 0x57, 0xa8, 0xb2, 0x47, 0x00, 0xd4, 0x9b, 0xea, 0x46, 0x10, 0x3a, 0xe8, 0xea, 0xc6, 0xa5, + 0xf4, 0xc0, 0xba, 0x2f, 0x9d, 0x1a, 0x6b, 0x56, 0x4b, 0x41, 0xf8, 0x98, 0x74, 0xa1, 0x7c, 0x00, + 0xdf, 0x20, 0xe5, 0xfc, 0x49, 0x2e, 0x72, 0xa1, 0x7c, 0x6c, 0x6d, 0x27, 0xf2, 0x49, 0x5f, 0xd5, + 0xf0, 0x22, 0x5f, 0xc1, 0x77, 0x22, 0x85, 0xaf, 0x48, 0x6b, 0x6f, 0xee, 0xac, 0xda, 0xc3, 0x4f, + 0xe1, 0xc6, 0x28, 0x97, 0x9a, 0x7b, 0xa8, 0xd9, 0xee, 0x0b, 0xed, 0x85, 0x1e, 0x10, 0x8f, 0x26, + 0xf4, 0xa2, 0xc0, 0x79, 0x2d, 0xcd, 0x73, 0xfb, 0x70, 0xc7, 0x7d, 0xf1, 0x49, 0x88, 0x93, 0xf4, + 0xdc, 0x63, 0xa2, 0xfa, 0x06, 0x59, 0xc4, 0x3f, 0xe7, 0x22, 0x7f, 0x1b, 0x0e, 0x33, 0xe5, 0xc9, + 0xbf, 0x36, 0xfe, 0x56, 0x7a, 0x15, 0x7f, 0xfb, 0xd5, 0x9b, 0x4f, 0xc2, 0xe1, 0x67, 0xc9, 0xf5, + 0x1b, 0x64, 0x41, 0x3f, 0x59, 0x87, 0xf2, 0xa6, 0xee, 0x13, 0x31, 0x5a, 0xbc, 0x21, 0xaa, 0x0f, + 0xfc, 0xb4, 0xea, 0x6a, 0x9a, 0x72, 0x02, 0x31, 0x7d, 0x77, 0x73, 0x41, 0xd4, 0x30, 0x44, 0xa5, + 0xf3, 0x5a, 0xe6, 0xc2, 0x55, 0x1c, 0x9e, 0x50, 0x43, 0x64, 0xfc, 0x11, 0x94, 0xa2, 0xda, 0x87, + 0x58, 0x96, 0xaf, 0x4e, 0xeb, 0x49, 0x4c, 0x35, 0xee, 0x40, 0x7b, 0x47, 0x3b, 0x02, 0xc2, 0xc2, + 0x56, 0xa7, 0x9f, 0x48, 0x54, 0xe3, 0x0e, 0xf8, 0x03, 0x28, 0x86, 0xd5, 0x47, 0x26, 0x92, 0x72, + 0xc6, 0x4d, 0xb1, 0x64, 0x2d, 0x56, 0x8d, 0xd0, 0xf1, 0x1d, 0x28, 0xf8, 0xc4, 0xe1, 0x1b, 0x2d, + 0xe5, 0x51, 0xe1, 0x27, 0x77, 0xbc, 0x19, 0x1a, 0xae, 0xc3, 0x22, 0xfd, 0xd5, 0x3c, 0xbe, 0x01, + 0x2e, 0xce, 0x4e, 0xac, 0x4d, 0xee, 0xc6, 0xf1, 0xd4, 0xb2, 0x9f, 0xd8, 0x35, 0xff, 0x36, 0x00, + 0x23, 0xc2, 0xd5, 0x5e, 0x9c, 0x36, 0xda, 0x70, 0x8f, 0x5a, 0x2d, 0xf9, 0xd1, 0x76, 0xf5, 0xa3, + 0x58, 0xff, 0xa5, 0x29, 0x1a, 0x0a, 0x0f, 0x36, 0x46, 0x3b, 0x4b, 0xb7, 0x21, 0xaf, 0x1b, 0xc7, + 0xe2, 0x44, 0xbf, 0x3c, 0x69, 0xb3, 0x53, 0xa5, 0x48, 0x54, 0x2c, 0x87, 0xb6, 0xfb, 0x82, 0x95, + 0xbd, 0x26, 0x89, 0xe5, 0x89, 0xed, 0xbe, 0x50, 0x19, 0x1a, 0xde, 0x84, 0xf2, 0x30, 0xde, 0xd2, + 0x11, 0x27, 0x51, 0xb3, 0xa5, 0x92, 0xd8, 0xfa, 0x51, 0x93, 0x9d, 0xe8, 0xb0, 0x7c, 0x5e, 0x23, + 0x67, 0xd5, 0xb2, 0x49, 0xc3, 0x12, 0x75, 0x74, 0x35, 0x44, 0xc6, 0xf7, 0xc2, 0xf9, 0x53, 0xcd, + 0x72, 0x4c, 0xc9, 0x22, 0x74, 0x38, 0x81, 0x9a, 0x50, 0x35, 0x6c, 0xd7, 0x27, 0xd1, 0x59, 0x15, + 0x56, 0x5c, 0x2b, 0x6f, 0x28, 0xd9, 0xf6, 0x9a, 0xdc, 0x5a, 0x51, 0x2b, 0x46, 0x6a, 0xa7, 0x25, + 0x22, 0x15, 0x16, 0x99, 0xd8, 0xf1, 0xd2, 0xa9, 0xa4, 0xc2, 0x52, 0xa3, 0x20, 0x15, 0x6d, 0xae, + 0xb4, 0xd9, 0xb1, 0x77, 0x5e, 0x5e, 0x0f, 0x05, 0xc1, 0x0f, 0x9c, 0xbe, 0x3e, 0xd5, 0x98, 0x43, + 0x81, 0x2c, 0x0d, 0x46, 0x76, 0x1a, 0xee, 0x40, 0x61, 0x60, 0x39, 0x3d, 0x76, 0xcf, 0x6e, 0x92, + 0x0e, 0xf7, 0x2c, 0xa7, 0xa7, 0x32, 0x34, 0x86, 0xee, 0x3a, 0x3d, 0x76, 0xe1, 0x6e, 0x22, 0xba, + 0xcb, 0xd0, 0x5d, 0xa7, 0x87, 0x7f, 0x13, 0xae, 0x7b, 0xd3, 0xf7, 0x70, 0xd8, 0xb5, 0xbc, 0xf2, + 0xc6, 0xc3, 0x4c, 0x4a, 0x33, 0xf6, 0x7f, 0xd4, 0x59, 0xc4, 0xf1, 0xaf, 0xc1, 0xf9, 0x68, 0x71, + 0x17, 0x9e, 0xb4, 0x94, 0x2f, 0xb0, 0x2f, 0xde, 0x79, 0xb5, 0xe3, 0x99, 0xe3, 0x74, 0xb0, 0x9f, + 0xb8, 0xe7, 0x35, 0x7a, 0x8c, 0x93, 0xdd, 0x23, 0x2c, 0x6f, 0xbc, 0xfb, 0x85, 0xce, 0x80, 0xaa, + 0x93, 0xe9, 0xd2, 0x49, 0x64, 0xc7, 0x67, 0xf6, 0xd8, 0x2d, 0xc4, 0x49, 0x93, 0x28, 0x79, 0xb6, + 0x2f, 0xd9, 0x09, 0x7f, 0x06, 0xcb, 0xf6, 0xf8, 0xb9, 0x3f, 0x76, 0x4b, 0xb1, 0x3c, 0xe1, 0x80, + 0x43, 0xc6, 0x39, 0x41, 0x35, 0x8b, 0x08, 0x7e, 0x16, 0x9f, 0xbb, 0x67, 0x5b, 0x1b, 0xec, 0x46, + 0xe3, 0x44, 0x53, 0x4f, 0x6d, 0x82, 0xa4, 0x3b, 0xe2, 0x1f, 0xc2, 0x05, 0x23, 0x6b, 0x93, 0x84, + 0xdd, 0x81, 0x2c, 0x6f, 0xdc, 0x3e, 0x03, 0xc5, 0x90, 0xd3, 0x6c, 0x42, 0xb8, 0x0b, 0xe7, 0xbd, + 0xd1, 0xfd, 0x54, 0x76, 0x7d, 0xb2, 0x3c, 0xe1, 0xbe, 0xc2, 0xd8, 0xee, 0xab, 0x3a, 0x4e, 0x80, + 0x07, 0x0b, 0x72, 0xcc, 0xee, 0x55, 0x4e, 0x0e, 0x16, 0xe4, 0x58, 0x65, 0x68, 0xf8, 0x7b, 0x80, + 0x7a, 0x23, 0xd5, 0x73, 0x76, 0xdb, 0xb2, 0xbc, 0x71, 0x6b, 0x52, 0x31, 0x38, 0x5d, 0x6a, 0x1f, + 0xeb, 0x8e, 0x2d, 0x90, 0x7b, 0x13, 0x0a, 0xf2, 0xec, 0x8e, 0xe6, 0x24, 0xe3, 0x9f, 0x54, 0xc5, + 0x57, 0x27, 0x92, 0xc3, 0x1a, 0x5c, 0xe4, 0xc7, 0x04, 0x22, 0xdf, 0xa6, 0x19, 0xec, 0x90, 0x81, + 0x7c, 0x9d, 0x7d, 0xe8, 0xad, 0x09, 0x11, 0x64, 0xfc, 0x54, 0x82, 0xba, 0xa2, 0x67, 0x9d, 0x55, + 0xf8, 0x01, 0xac, 0xf4, 0x32, 0x6a, 0xd2, 0xf2, 0xda, 0x14, 0xf2, 0x99, 0x45, 0xec, 0x4c, 0x32, + 0x78, 0x08, 0xd7, 0x7a, 0x53, 0x4a, 0xde, 0xf2, 0x0d, 0xf6, 0x99, 0xfb, 0x67, 0xff, 0x4c, 0x28, + 0xb2, 0xa9, 0x64, 0x69, 0x26, 0xd3, 0x0b, 0x4b, 0xd3, 0xb2, 0x32, 0x25, 0xb6, 0xc7, 0x05, 0xec, + 0xb8, 0x03, 0xb5, 0xdb, 0xde, 0x68, 0x61, 0x5b, 0xbe, 0x39, 0xc5, 0x6e, 0xc7, 0xca, 0xe0, 0xea, + 0x38, 0x01, 0x3a, 0x73, 0xf5, 0xe4, 0xfd, 0x2d, 0xf9, 0xf5, 0x29, 0x33, 0x37, 0x75, 0xd3, 0x4b, + 0x4d, 0x77, 0xc4, 0x0d, 0x58, 0xd4, 0x13, 0x57, 0xd5, 0xe4, 0x5b, 0x8c, 0xd0, 0x8d, 0x89, 0x84, + 0x22, 0xae, 0x52, 0xdd, 0xa8, 0xab, 0xd3, 0xe3, 0x73, 0x3b, 0xf2, 0x1b, 0x53, 0x5c, 0x5d, 0xe2, + 0x7c, 0x8f, 0x9a, 0xec, 0x24, 0x44, 0x95, 0x2e, 0x4a, 0xcb, 0x6f, 0x4e, 0x17, 0xd5, 0x48, 0x09, + 0x7b, 0x9c, 0x00, 0xb6, 0xe1, 0x72, 0x6f, 0x52, 0xa9, 0x5b, 0x5e, 0x67, 0xd4, 0xef, 0x9e, 0x91, + 0x7a, 0xe4, 0xf2, 0x27, 0x12, 0xc4, 0x0f, 0x60, 0xde, 0x61, 0xb5, 0x61, 0x79, 0x23, 0x6b, 0x23, + 0x2b, 0x5d, 0x3e, 0x16, 0xa8, 0x78, 0x1b, 0xaa, 0x4e, 0xaa, 0xa0, 0x2c, 0x3f, 0x60, 0x9d, 0x6f, + 0x4e, 0xeb, 0x1c, 0x32, 0x33, 0xd2, 0x95, 0x4a, 0x51, 0x1f, 0xad, 0x86, 0xca, 0x0f, 0xa7, 0x48, + 0x71, 0xbc, 0x76, 0x3a, 0x4e, 0x80, 0x4a, 0x51, 0x9f, 0x54, 0x63, 0x95, 0xdf, 0x9d, 0x22, 0xc5, + 0x89, 0x95, 0x59, 0x75, 0x32, 0x41, 0xea, 0x48, 0xf4, 0x8c, 0x4a, 0x9e, 0xfc, 0x68, 0x9a, 0x9f, + 0xca, 0x2a, 0xfd, 0x65, 0x92, 0xa1, 0x8e, 0x44, 0x9f, 0x52, 0x28, 0x94, 0xdf, 0x9b, 0xe2, 0x48, + 0xa6, 0x55, 0x18, 0xd5, 0xa9, 0x64, 0xa9, 0x6d, 0x10, 0xb6, 0x5c, 0x95, 0xdf, 0x9f, 0x62, 0x1b, + 0xa2, 0x2e, 0x26, 0x50, 0xa9, 0x6d, 0x90, 0x54, 0xa5, 0x4c, 0xfe, 0x60, 0x8a, 0x6d, 0xa4, 0x8b, + 0x6a, 0xea, 0x48, 0x57, 0x6a, 0x1b, 0x64, 0xb4, 0xde, 0x22, 0x3f, 0x9e, 0x62, 0x1b, 0xe3, 0xd5, + 0x99, 0x71, 0x02, 0xd4, 0x36, 0xc8, 0xa4, 0x2a, 0x8e, 0xfc, 0xe1, 0x14, 0xdb, 0x98, 0x58, 0xfb, + 0x51, 0x27, 0x13, 0xa4, 0xb6, 0x41, 0x32, 0x16, 0xfd, 0xf2, 0x47, 0x53, 0x6c, 0x23, 0xb3, 0x4a, + 0x90, 0x49, 0x86, 0xda, 0x06, 0x99, 0x52, 0x53, 0x90, 0xbf, 0x3d, 0xc5, 0x36, 0xa6, 0x15, 0x23, + 0xd4, 0xa9, 0x64, 0x95, 0x9f, 0x15, 0xc5, 0x1f, 0x25, 0x95, 0x61, 0xa1, 0xde, 0x6e, 0xb5, 0x1a, + 0xf5, 0x2e, 0xca, 0xe1, 0x0a, 0x94, 0xc4, 0x4b, 0x63, 0x0b, 0xe5, 0xe9, 0x6b, 0x67, 0x7f, 0xb3, + 0x53, 0x57, 0x9b, 0x9b, 0x0d, 0x54, 0x60, 0xff, 0x99, 0xa4, 0xb6, 0xb7, 0xf6, 0xeb, 0x0d, 0x95, + 0xff, 0x3f, 0x52, 0xa7, 0xd1, 0xda, 0x42, 0xf3, 0x18, 0xc1, 0x22, 0x7d, 0xd2, 0xd4, 0x46, 0xbd, + 0xd1, 0xdc, 0xeb, 0xa2, 0x05, 0x5c, 0x05, 0x60, 0x90, 0x86, 0xaa, 0xb6, 0x55, 0x54, 0xa4, 0x1f, + 0xd9, 0x6d, 0x74, 0x3a, 0xb5, 0xa7, 0x0d, 0x54, 0x62, 0x3b, 0xcb, 0xf5, 0x6d, 0x04, 0x94, 0xc2, + 0x93, 0x9d, 0xf6, 0x27, 0xa8, 0x8c, 0x97, 0xa0, 0xbc, 0xdf, 0x8a, 0x3f, 0xb5, 0xc8, 0xae, 0x1e, + 0xed, 0xd7, 0xeb, 0x8d, 0x4e, 0x07, 0x55, 0x70, 0x09, 0xe6, 0x38, 0xa1, 0x2a, 0xc6, 0x50, 0xad, + 0xef, 0xb4, 0x3b, 0x0d, 0x2d, 0x62, 0x64, 0x29, 0x86, 0xd5, 0xdb, 0xad, 0xce, 0xfe, 0x6e, 0x43, + 0x45, 0x08, 0xaf, 0x00, 0x0a, 0x31, 0xb4, 0x90, 0xd0, 0x79, 0xfa, 0xc1, 0xbd, 0x66, 0xeb, 0x29, + 0xc2, 0xec, 0xa9, 0xdd, 0x7a, 0x8a, 0x96, 0xf1, 0x2d, 0xb8, 0xa1, 0x36, 0xb6, 0x1a, 0x3b, 0xcd, + 0xe7, 0x0d, 0x55, 0xdb, 0x6f, 0xd5, 0xea, 0xdb, 0xad, 0xf6, 0x27, 0x3b, 0x8d, 0xad, 0xa7, 0x8d, + 0x2d, 0x4d, 0xf0, 0xdc, 0x41, 0x2b, 0x58, 0x86, 0x95, 0xbd, 0x9a, 0xda, 0x6d, 0x76, 0x9b, 0xed, + 0x16, 0x6b, 0xe9, 0xd6, 0xb6, 0x6a, 0xdd, 0x1a, 0xba, 0x80, 0x6f, 0xc0, 0x6b, 0x59, 0x2d, 0x9a, + 0xda, 0xe8, 0xec, 0xb5, 0x5b, 0x9d, 0x06, 0xba, 0xc8, 0x2e, 0xdb, 0xb5, 0xdb, 0xdb, 0xfb, 0x7b, + 0xe8, 0x12, 0x5e, 0x86, 0x25, 0xfe, 0x1c, 0x23, 0xc8, 0x6c, 0x08, 0x82, 0x79, 0xad, 0xd3, 0xad, + 0x75, 0x3b, 0xe8, 0x32, 0xbe, 0x0a, 0x97, 0xd2, 0xb0, 0xb8, 0xc3, 0x15, 0xca, 0x8e, 0xda, 0xa8, + 0xd5, 0x9f, 0x35, 0xb6, 0x34, 0x2a, 0xe7, 0xf6, 0x13, 0xad, 0xdb, 0xde, 0x6b, 0xd6, 0xd1, 0x55, + 0xae, 0x96, 0xc6, 0x36, 0xba, 0x86, 0x2f, 0xc1, 0xf2, 0xd3, 0x46, 0x57, 0xdb, 0xa9, 0x75, 0xba, + 0xe1, 0x48, 0xb4, 0xe6, 0x16, 0x7a, 0x0d, 0xaf, 0xc1, 0xb5, 0x8c, 0x86, 0x98, 0xfc, 0x2a, 0xbe, + 0x02, 0x17, 0x6b, 0xf5, 0x6e, 0xf3, 0x79, 0x2c, 0x53, 0xad, 0xfe, 0xac, 0xd6, 0x7a, 0xda, 0x40, + 0xd7, 0x29, 0x5f, 0xb4, 0x37, 0xfb, 0x5e, 0x87, 0x7e, 0xb9, 0x55, 0xdb, 0x6d, 0x74, 0xf6, 0x6a, + 0xf5, 0x06, 0x5a, 0xc3, 0xaf, 0xc3, 0xda, 0x84, 0xc6, 0x98, 0xfc, 0x0d, 0x6a, 0x1e, 0x14, 0xab, + 0x53, 0x7f, 0xd6, 0xd8, 0xad, 0x21, 0x25, 0xe4, 0x94, 0xbf, 0xc7, 0x88, 0x37, 0xa9, 0x5c, 0x6a, + 0xfb, 0xdd, 0x67, 0xf4, 0xe3, 0x3b, 0x3b, 0x0d, 0xfa, 0xfd, 0xd7, 0xf1, 0x79, 0xa8, 0x30, 0x58, + 0x84, 0x76, 0x8b, 0x1a, 0x60, 0xad, 0xbe, 0x1d, 0x43, 0xde, 0xa0, 0xf2, 0xa1, 0x14, 0xdb, 0xaa, + 0x56, 0x57, 0x1b, 0xb5, 0x6e, 0x23, 0xfc, 0xd6, 0x9b, 0x54, 0x5d, 0x59, 0x2d, 0x71, 0xe7, 0x75, + 0x6a, 0x7c, 0xad, 0xc6, 0x27, 0x5a, 0xf7, 0x57, 0x5b, 0x68, 0x83, 0x5a, 0x92, 0x78, 0x89, 0x51, + 0x1e, 0x50, 0xfa, 0xb5, 0xad, 0x2d, 0x2d, 0x52, 0xbc, 0xd6, 0x6d, 0x33, 0xfc, 0x87, 0x94, 0x7e, + 0x56, 0x4b, 0xdc, 0xf9, 0x5d, 0x2a, 0x41, 0x8a, 0x22, 0xec, 0x7d, 0x2f, 0xd9, 0xff, 0x11, 0x95, + 0xe0, 0x84, 0xc6, 0x98, 0xc4, 0x7b, 0x94, 0x45, 0xaa, 0x77, 0xda, 0xe5, 0x7d, 0xca, 0xa2, 0x78, + 0x89, 0x51, 0x3e, 0xa0, 0x2c, 0x86, 0xd0, 0x76, 0x2b, 0xe6, 0x07, 0x3d, 0xa6, 0x2c, 0x66, 0xb5, + 0xc4, 0x9d, 0x3f, 0xa4, 0x2c, 0x26, 0x50, 0x92, 0xcc, 0xa0, 0x8f, 0x28, 0x8b, 0x13, 0x1a, 0x63, + 0x12, 0xdf, 0xbe, 0xbd, 0xc5, 0x6e, 0x41, 0x25, 0xff, 0x5e, 0x89, 0xfd, 0x29, 0x5b, 0xbb, 0xd5, + 0x40, 0xe7, 0xa8, 0x0f, 0xd8, 0xf9, 0xec, 0x21, 0xff, 0x47, 0xb6, 0xcf, 0x76, 0x9a, 0x9b, 0x28, + 0xc7, 0x9e, 0x3a, 0x5d, 0xea, 0x76, 0x00, 0xe6, 0x3b, 0xad, 0xda, 0xde, 0xde, 0xa7, 0xa8, 0x70, + 0xbb, 0x01, 0x78, 0xfc, 0xba, 0x4e, 0xe2, 0xea, 0xeb, 0xb9, 0xf4, 0xad, 0x58, 0x76, 0x3c, 0xfc, + 0x13, 0xdd, 0x0a, 0x9e, 0xb8, 0x5e, 0x0c, 0xcd, 0xdd, 0xfe, 0xad, 0x39, 0x28, 0x27, 0x0a, 0xa1, + 0xd4, 0x62, 0xf6, 0x9d, 0x63, 0xc7, 0x7d, 0x21, 0x0e, 0x84, 0x9f, 0xa3, 0x66, 0x15, 0xae, 0xa7, + 0x13, 0x27, 0xcd, 0xf7, 0x88, 0xe7, 0xb3, 0x7b, 0x7b, 0x86, 0x38, 0x4e, 0x9e, 0xa3, 0xc6, 0x4a, + 0xf3, 0x52, 0xe2, 0x04, 0x96, 0x11, 0x1f, 0x67, 0x47, 0x79, 0x7c, 0x11, 0x70, 0x8d, 0x5f, 0x65, + 0xfb, 0x3c, 0x01, 0x2f, 0xd0, 0x6f, 0x85, 0xeb, 0x96, 0xcd, 0xa1, 0x7f, 0x8a, 0xe6, 0xa8, 0x0f, + 0x10, 0x97, 0xcc, 0x5a, 0x6e, 0xc0, 0x4e, 0x3b, 0xa2, 0x79, 0xea, 0x88, 0xc2, 0x91, 0x6e, 0xf2, + 0x23, 0x65, 0xdf, 0x1b, 0xba, 0x81, 0xde, 0x78, 0x69, 0x10, 0x62, 0x12, 0x5e, 0x7f, 0x42, 0x0b, + 0xf8, 0x2d, 0xb8, 0x35, 0x15, 0xed, 0xa5, 0x41, 0xf8, 0x09, 0xfa, 0x22, 0x1d, 0x52, 0x78, 0x52, + 0x9e, 0xf7, 0x2e, 0x51, 0xbd, 0xee, 0x3b, 0xe2, 0x5f, 0x2e, 0x88, 0x29, 0xce, 0x36, 0xf0, 0x46, + 0xa0, 0xf8, 0x6c, 0x55, 0xd2, 0x72, 0x83, 0x27, 0xee, 0xd0, 0x31, 0x51, 0x99, 0x1a, 0x51, 0x32, + 0x7c, 0x44, 0x2d, 0x8b, 0xec, 0x18, 0x7e, 0x78, 0x06, 0x2f, 0x84, 0x56, 0xe8, 0xc8, 0xba, 0xae, + 0xbb, 0xab, 0x3b, 0xa7, 0x2a, 0x2f, 0x7b, 0xfb, 0xa8, 0x4a, 0x89, 0x30, 0xba, 0x5d, 0xe2, 0xf5, + 0x2d, 0x47, 0x0f, 0xc2, 0xc1, 0x2c, 0x51, 0xd1, 0x44, 0x83, 0xa1, 0xa2, 0x61, 0x8e, 0xbb, 0xe9, + 0xb0, 0xcb, 0x11, 0x9c, 0x15, 0xbd, 0x4f, 0xd0, 0x79, 0x2a, 0xda, 0x26, 0xbb, 0x2b, 0xa0, 0x07, + 0xd6, 0x81, 0x2d, 0x72, 0x60, 0x84, 0xa9, 0x2e, 0x42, 0x26, 0x6a, 0xbe, 0x6f, 0xf5, 0xc4, 0x50, + 0x96, 0xb1, 0x02, 0xab, 0x5d, 0x4f, 0x77, 0x7c, 0xbe, 0x67, 0x50, 0x77, 0x5d, 0xcf, 0xa4, 0x5f, + 0x76, 0x63, 0x5e, 0x57, 0x92, 0x9f, 0x7a, 0xc9, 0x2e, 0x8e, 0x0f, 0x7d, 0x74, 0x81, 0x8e, 0xa0, + 0xe5, 0x06, 0x35, 0xdb, 0x76, 0x5f, 0x84, 0x7c, 0x5e, 0xa4, 0xdf, 0x49, 0x91, 0x73, 0x0e, 0x6d, + 0xcb, 0x08, 0xd0, 0xa5, 0x91, 0x86, 0x88, 0xb8, 0x2c, 0xfe, 0x8c, 0x8f, 0x8d, 0xec, 0x09, 0xb5, + 0x1e, 0x13, 0x5d, 0xbe, 0xbd, 0x0d, 0x90, 0xb8, 0x3c, 0x4a, 0x7d, 0x5b, 0xfc, 0x97, 0x12, 0xfc, + 0xbf, 0x0a, 0x97, 0x61, 0x29, 0x86, 0x7d, 0x6a, 0xe8, 0xcf, 0xef, 0x73, 0x33, 0x8c, 0x81, 0x35, + 0x6a, 0x79, 0x3e, 0xca, 0xdd, 0xfe, 0x53, 0x09, 0x96, 0xf6, 0x46, 0xfe, 0x38, 0x64, 0x1e, 0x72, + 0x27, 0xf7, 0xd0, 0x39, 0xf6, 0x4b, 0x7b, 0xd2, 0xdf, 0x0d, 0x94, 0x63, 0xbf, 0x0f, 0x50, 0x9e, + 0xfd, 0x3e, 0x44, 0x05, 0xf6, 0xfb, 0x2e, 0x9a, 0x63, 0xbf, 0x8f, 0xd0, 0x3c, 0xfb, 0x7d, 0x0f, + 0x2d, 0xb0, 0xdf, 0xf7, 0x51, 0x91, 0xfd, 0x7e, 0xc0, 0x23, 0xf5, 0xc9, 0xfd, 0x7b, 0x08, 0xf8, + 0xc3, 0x7d, 0x54, 0xe6, 0x0f, 0x1b, 0x68, 0x91, 0x3f, 0x3c, 0x40, 0x15, 0xfe, 0xf0, 0x10, 0x55, + 0xf9, 0xc3, 0xbb, 0x68, 0x89, 0x3f, 0x3c, 0x42, 0x88, 0x3f, 0xbc, 0x87, 0xce, 0xdf, 0x7e, 0x3b, + 0xf9, 0x6f, 0x18, 0xe2, 0x00, 0x5a, 0x6d, 0xbf, 0xdb, 0xd6, 0x3a, 0x7b, 0x3b, 0xcd, 0xae, 0xb8, + 0xca, 0xde, 0x6d, 0xd6, 0xb7, 0x3f, 0x45, 0xd2, 0x6d, 0x05, 0x4a, 0xd1, 0x6e, 0x0e, 0x6d, 0xa8, + 0xb7, 0x77, 0x77, 0x19, 0x52, 0x09, 0xe6, 0x6a, 0x9b, 0x6d, 0xb5, 0x8b, 0xa4, 0xcd, 0x8d, 0x9f, + 0xfc, 0x7c, 0x55, 0xfa, 0x87, 0x9f, 0xaf, 0x4a, 0xff, 0xf2, 0xf3, 0x55, 0x09, 0x14, 0xd7, 0xeb, + 0xdd, 0xd5, 0x07, 0xba, 0x71, 0x44, 0xc2, 0x34, 0xc9, 0x70, 0xfb, 0x7d, 0xd7, 0xb9, 0xab, 0x87, + 0xff, 0x66, 0xf9, 0x2c, 0xff, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xe0, 0x3f, 0x51, 0x56, 0xe1, + 0x52, 0x00, 0x00, } func (m *Schema) Marshal() (dAtA []byte, err error) { @@ -6772,6 +7180,18 @@ func (m *MessageIdData) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.BatchSize != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.BatchSize)) + i-- + dAtA[i] = 0x30 + } + if len(m.AckSet) > 0 { + for iNdEx := len(m.AckSet) - 1; iNdEx >= 0; iNdEx-- { + i = encodeVarintPulsarApi(dAtA, i, uint64(m.AckSet[iNdEx])) + i-- + dAtA[i] = 0x28 + } + } if m.BatchIndex != nil { i = encodeVarintPulsarApi(dAtA, i, uint64(*m.BatchIndex)) i-- @@ -7011,6 +7431,60 @@ func (m *MessageMetadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.NullPartitionKey != nil { + i-- + if *m.NullPartitionKey { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xf0 + } + if m.ChunkId != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.ChunkId)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xe8 + } + if m.TotalChunkMsgSize != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.TotalChunkMsgSize)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xe0 + } + if m.NumChunksFromMsg != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.NumChunksFromMsg)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xd8 + } + if m.Uuid != nil { + i -= len(*m.Uuid) + copy(dAtA[i:], *m.Uuid) + i = encodeVarintPulsarApi(dAtA, i, uint64(len(*m.Uuid))) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xd2 + } + if m.NullValue != nil { + i-- + if *m.NullValue { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xc8 + } if m.HighestSequenceId != nil { i = encodeVarintPulsarApi(dAtA, i, uint64(*m.HighestSequenceId)) i-- @@ -7211,6 +7685,26 @@ func (m *SingleMessageMetadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.NullPartitionKey != nil { + i-- + if *m.NullPartitionKey { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x50 + } + if m.NullValue != nil { + i-- + if *m.NullValue { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x48 + } if m.SequenceId != nil { i = encodeVarintPulsarApi(dAtA, i, uint64(*m.SequenceId)) i-- @@ -7279,6 +7773,43 @@ func (m *SingleMessageMetadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *BrokerEntryMetadata) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BrokerEntryMetadata) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BrokerEntryMetadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Index != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.Index)) + i-- + dAtA[i] = 0x10 + } + if m.BrokerTimestamp != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.BrokerTimestamp)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func (m *CommandConnect) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -7403,6 +7934,16 @@ func (m *FeatureFlags) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.SupportsBrokerEntryMetadata != nil { + i-- + if *m.SupportsBrokerEntryMetadata { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } if m.SupportsAuthRefresh != nil { i-- if *m.SupportsAuthRefresh { @@ -8155,9 +8696,19 @@ func (m *CommandProducer) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } - if m.UserProvidedProducerName != nil { + if m.TopicEpoch != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.TopicEpoch)) i-- - if *m.UserProvidedProducerName { + dAtA[i] = 0x58 + } + if m.ProducerAccessMode != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.ProducerAccessMode)) + i-- + dAtA[i] = 0x50 + } + if m.UserProvidedProducerName != nil { + i-- + if *m.UserProvidedProducerName { dAtA[i] = 1 } else { dAtA[i] = 0 @@ -8263,6 +8814,16 @@ func (m *CommandSend) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.IsChunk != nil { + i-- + if *m.IsChunk { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x38 + } if m.HighestSequenceId != nil { i = encodeVarintPulsarApi(dAtA, i, uint64(*m.HighestSequenceId)) i-- @@ -8439,6 +9000,13 @@ func (m *CommandMessage) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.AckSet) > 0 { + for iNdEx := len(m.AckSet) - 1; iNdEx >= 0; iNdEx-- { + i = encodeVarintPulsarApi(dAtA, i, uint64(m.AckSet[iNdEx])) + i-- + dAtA[i] = 0x20 + } + } if m.RedeliveryCount != nil { i = encodeVarintPulsarApi(dAtA, i, uint64(*m.RedeliveryCount)) i-- @@ -8492,6 +9060,11 @@ func (m *CommandAck) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.RequestId != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.RequestId)) + i-- + dAtA[i] = 0x40 + } if m.TxnidMostBits != nil { i = encodeVarintPulsarApi(dAtA, i, uint64(*m.TxnidMostBits)) i-- @@ -8576,6 +9149,11 @@ func (m *CommandAckResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.RequestId != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.RequestId)) + i-- + dAtA[i] = 0x30 + } if m.Message != nil { i -= len(*m.Message) copy(dAtA[i:], *m.Message) @@ -9026,6 +9604,21 @@ func (m *CommandProducerSuccess) MarshalToSizedBuffer(dAtA []byte) (int, error) i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.ProducerReady != nil { + i-- + if *m.ProducerReady { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x30 + } + if m.TopicEpoch != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.TopicEpoch)) + i-- + dAtA[i] = 0x28 + } if m.SchemaVersion != nil { i -= len(m.SchemaVersion) copy(dAtA[i:], m.SchemaVersion) @@ -9390,6 +9983,18 @@ func (m *CommandGetLastMessageIdResponse) MarshalToSizedBuffer(dAtA []byte) (int i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.ConsumerMarkDeletePosition != nil { + { + size, err := m.ConsumerMarkDeletePosition.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPulsarApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } if m.RequestId == nil { return 0, github_com_gogo_protobuf_proto.NewRequiredNotSetError("request_id") } else { @@ -10227,6 +10832,11 @@ func (m *CommandEndTxnOnPartition) MarshalToSizedBuffer(dAtA []byte) (int, error i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.TxnidLeastBitsOfLowWatermark != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.TxnidLeastBitsOfLowWatermark)) + i-- + dAtA[i] = 0x30 + } if m.TxnAction != nil { i = encodeVarintPulsarApi(dAtA, i, uint64(*m.TxnAction)) i-- @@ -10339,6 +10949,11 @@ func (m *CommandEndTxnOnSubscription) MarshalToSizedBuffer(dAtA []byte) (int, er i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.TxnidLeastBitsOfLowWatermark != nil { + i = encodeVarintPulsarApi(dAtA, i, uint64(*m.TxnidLeastBitsOfLowWatermark)) + i-- + dAtA[i] = 0x30 + } if m.TxnAction != nil { i = encodeVarintPulsarApi(dAtA, i, uint64(*m.TxnAction)) i-- @@ -11210,6 +11825,14 @@ func (m *MessageIdData) Size() (n int) { if m.BatchIndex != nil { n += 1 + sovPulsarApi(uint64(*m.BatchIndex)) } + if len(m.AckSet) > 0 { + for _, e := range m.AckSet { + n += 1 + sovPulsarApi(uint64(e)) + } + } + if m.BatchSize != nil { + n += 1 + sovPulsarApi(uint64(*m.BatchSize)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -11387,6 +12010,25 @@ func (m *MessageMetadata) Size() (n int) { if m.HighestSequenceId != nil { n += 2 + sovPulsarApi(uint64(*m.HighestSequenceId)) } + if m.NullValue != nil { + n += 3 + } + if m.Uuid != nil { + l = len(*m.Uuid) + n += 2 + l + sovPulsarApi(uint64(l)) + } + if m.NumChunksFromMsg != nil { + n += 2 + sovPulsarApi(uint64(*m.NumChunksFromMsg)) + } + if m.TotalChunkMsgSize != nil { + n += 2 + sovPulsarApi(uint64(*m.TotalChunkMsgSize)) + } + if m.ChunkId != nil { + n += 2 + sovPulsarApi(uint64(*m.ChunkId)) + } + if m.NullPartitionKey != nil { + n += 3 + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -11428,6 +12070,30 @@ func (m *SingleMessageMetadata) Size() (n int) { if m.SequenceId != nil { n += 1 + sovPulsarApi(uint64(*m.SequenceId)) } + if m.NullValue != nil { + n += 2 + } + if m.NullPartitionKey != nil { + n += 2 + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *BrokerEntryMetadata) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BrokerTimestamp != nil { + n += 1 + sovPulsarApi(uint64(*m.BrokerTimestamp)) + } + if m.Index != nil { + n += 1 + sovPulsarApi(uint64(*m.Index)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -11493,6 +12159,9 @@ func (m *FeatureFlags) Size() (n int) { if m.SupportsAuthRefresh != nil { n += 2 } + if m.SupportsBrokerEntryMetadata != nil { + n += 2 + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -11858,6 +12527,12 @@ func (m *CommandProducer) Size() (n int) { if m.UserProvidedProducerName != nil { n += 2 } + if m.ProducerAccessMode != nil { + n += 1 + sovPulsarApi(uint64(*m.ProducerAccessMode)) + } + if m.TopicEpoch != nil { + n += 1 + sovPulsarApi(uint64(*m.TopicEpoch)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -11888,6 +12563,9 @@ func (m *CommandSend) Size() (n int) { if m.HighestSequenceId != nil { n += 1 + sovPulsarApi(uint64(*m.HighestSequenceId)) } + if m.IsChunk != nil { + n += 2 + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -11960,6 +12638,11 @@ func (m *CommandMessage) Size() (n int) { if m.RedeliveryCount != nil { n += 1 + sovPulsarApi(uint64(*m.RedeliveryCount)) } + if len(m.AckSet) > 0 { + for _, e := range m.AckSet { + n += 1 + sovPulsarApi(uint64(e)) + } + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -11999,6 +12682,9 @@ func (m *CommandAck) Size() (n int) { if m.TxnidMostBits != nil { n += 1 + sovPulsarApi(uint64(*m.TxnidMostBits)) } + if m.RequestId != nil { + n += 1 + sovPulsarApi(uint64(*m.RequestId)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -12027,6 +12713,9 @@ func (m *CommandAckResponse) Size() (n int) { l = len(*m.Message) n += 1 + l + sovPulsarApi(uint64(l)) } + if m.RequestId != nil { + n += 1 + sovPulsarApi(uint64(*m.RequestId)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -12223,6 +12912,12 @@ func (m *CommandProducerSuccess) Size() (n int) { l = len(m.SchemaVersion) n += 1 + l + sovPulsarApi(uint64(l)) } + if m.TopicEpoch != nil { + n += 1 + sovPulsarApi(uint64(*m.TopicEpoch)) + } + if m.ProducerReady != nil { + n += 2 + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -12386,6 +13081,10 @@ func (m *CommandGetLastMessageIdResponse) Size() (n int) { if m.RequestId != nil { n += 1 + sovPulsarApi(uint64(*m.RequestId)) } + if m.ConsumerMarkDeletePosition != nil { + l = m.ConsumerMarkDeletePosition.Size() + n += 1 + l + sovPulsarApi(uint64(l)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -12790,6 +13489,9 @@ func (m *CommandEndTxnOnPartition) Size() (n int) { if m.TxnAction != nil { n += 1 + sovPulsarApi(uint64(*m.TxnAction)) } + if m.TxnidLeastBitsOfLowWatermark != nil { + n += 1 + sovPulsarApi(uint64(*m.TxnidLeastBitsOfLowWatermark)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -12846,6 +13548,9 @@ func (m *CommandEndTxnOnSubscription) Size() (n int) { if m.TxnAction != nil { n += 1 + sovPulsarApi(uint64(*m.TxnAction)) } + if m.TxnidLeastBitsOfLowWatermark != nil { + n += 1 + sovPulsarApi(uint64(*m.TxnidLeastBitsOfLowWatermark)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -13265,10 +13970,7 @@ func (m *Schema) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -13405,16 +14107,109 @@ func (m *MessageIdData) Unmarshal(dAtA []byte) error { } } m.BatchIndex = &v + case 5: + if wireType == 0 { + var v int64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.AckSet = append(m.AckSet, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthPulsarApi + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthPulsarApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.AckSet) == 0 { + m.AckSet = make([]int64, 0, elementCount) + } + for iNdEx < postIndex { + var v int64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.AckSet = append(m.AckSet, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field AckSet", wireType) + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BatchSize", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.BatchSize = &v default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -13540,10 +14335,7 @@ func (m *KeyValue) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -13656,10 +14448,7 @@ func (m *KeyLongValue) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -13759,10 +14548,7 @@ func (m *IntRange) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -13923,10 +14709,7 @@ func (m *EncryptionKeys) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -14556,16 +15339,148 @@ func (m *MessageMetadata) Unmarshal(dAtA []byte) error { } } m.HighestSequenceId = &v + case 25: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NullValue", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.NullValue = &b + case 26: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Uuid", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthPulsarApi + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthPulsarApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + s := string(dAtA[iNdEx:postIndex]) + m.Uuid = &s + iNdEx = postIndex + case 27: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NumChunksFromMsg", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NumChunksFromMsg = &v + case 28: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalChunkMsgSize", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TotalChunkMsgSize = &v + case 29: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunkId", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ChunkId = &v + case 30: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NullPartitionKey", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.NullPartitionKey = &b default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -14824,16 +15739,55 @@ func (m *SingleMessageMetadata) Unmarshal(dAtA []byte) error { } } m.SequenceId = &v + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NullValue", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.NullValue = &b + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NullPartitionKey", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.NullPartitionKey = &b default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -14852,6 +15806,97 @@ func (m *SingleMessageMetadata) Unmarshal(dAtA []byte) error { } return nil } +func (m *BrokerEntryMetadata) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BrokerEntryMetadata: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BrokerEntryMetadata: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BrokerTimestamp", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.BrokerTimestamp = &v + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Index = &v + default: + iNdEx = preIndex + skippy, err := skipPulsarApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthPulsarApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *CommandConnect) Unmarshal(dAtA []byte) error { var hasFields [1]uint64 l := len(dAtA) @@ -15197,10 +16242,7 @@ func (m *CommandConnect) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -15269,16 +16311,34 @@ func (m *FeatureFlags) Unmarshal(dAtA []byte) error { } b := bool(v != 0) m.SupportsAuthRefresh = &b + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SupportsBrokerEntryMetadata", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.SupportsBrokerEntryMetadata = &b default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -15404,10 +16464,7 @@ func (m *CommandConnected) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -15550,10 +16607,7 @@ func (m *CommandAuthResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -15693,10 +16747,7 @@ func (m *CommandAuthChallenge) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -15814,10 +16865,7 @@ func (m *AuthData) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -15945,10 +16993,7 @@ func (m *KeySharedMeta) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -16453,10 +17498,7 @@ func (m *CommandSubscribe) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -16677,10 +17719,7 @@ func (m *CommandPartitionedTopicMetadata) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -16852,10 +17891,7 @@ func (m *CommandPartitionedTopicMetadataResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -17118,10 +18154,7 @@ func (m *CommandLookupTopic) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -17381,10 +18414,7 @@ func (m *CommandLookupTopicResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -17672,18 +18702,55 @@ func (m *CommandProducer) Unmarshal(dAtA []byte) error { break } } - b := bool(v != 0) - m.UserProvidedProducerName = &b + b := bool(v != 0) + m.UserProvidedProducerName = &b + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProducerAccessMode", wireType) + } + var v ProducerAccessMode + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= ProducerAccessMode(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ProducerAccessMode = &v + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TopicEpoch", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TopicEpoch = &v default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -17860,16 +18927,34 @@ func (m *CommandSend) Unmarshal(dAtA []byte) error { } } m.HighestSequenceId = &v + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsChunk", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.IsChunk = &b default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -18025,10 +19110,7 @@ func (m *CommandSendReceipt) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -18183,10 +19265,7 @@ func (m *CommandSendError) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -18322,16 +19401,89 @@ func (m *CommandMessage) Unmarshal(dAtA []byte) error { } } m.RedeliveryCount = &v + case 4: + if wireType == 0 { + var v int64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.AckSet = append(m.AckSet, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthPulsarApi + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthPulsarApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.AckSet) == 0 { + m.AckSet = make([]int64, 0, elementCount) + } + for iNdEx < postIndex { + var v int64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.AckSet = append(m.AckSet, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field AckSet", wireType) + } default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -18553,16 +19705,33 @@ func (m *CommandAck) Unmarshal(dAtA []byte) error { } } m.TxnidMostBits = &v + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestId", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.RequestId = &v default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -18728,16 +19897,33 @@ func (m *CommandAckResponse) Unmarshal(dAtA []byte) error { s := string(dAtA[iNdEx:postIndex]) m.Message = &s iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestId", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.RequestId = &v default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -18834,10 +20020,7 @@ func (m *CommandActiveConsumerChange) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -18934,10 +20117,7 @@ func (m *CommandFlow) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -19037,10 +20217,7 @@ func (m *CommandUnsubscribe) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -19196,10 +20373,7 @@ func (m *CommandSeek) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -19278,10 +20452,7 @@ func (m *CommandReachedEndOfTopic) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -19378,10 +20549,7 @@ func (m *CommandCloseProducer) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -19481,10 +20649,7 @@ func (m *CommandCloseConsumer) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -19597,10 +20762,7 @@ func (m *CommandRedeliverUnacknowledgedMessages) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -19712,10 +20874,7 @@ func (m *CommandSuccess) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -19873,16 +21032,54 @@ func (m *CommandProducerSuccess) Unmarshal(dAtA []byte) error { m.SchemaVersion = []byte{} } iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TopicEpoch", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TopicEpoch = &v + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProducerReady", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.ProducerReady = &b default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -20016,10 +21213,7 @@ func (m *CommandError) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -20079,10 +21273,7 @@ func (m *CommandPing) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -20133,10 +21324,7 @@ func (m *CommandPong) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -20230,10 +21418,7 @@ func (m *CommandConsumerStats) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -20626,10 +21811,7 @@ func (m *CommandConsumerStatsResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -20726,10 +21908,7 @@ func (m *CommandGetLastMessageId) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -20839,16 +22018,49 @@ func (m *CommandGetLastMessageIdResponse) Unmarshal(dAtA []byte) error { } m.RequestId = &v hasFields[0] |= uint64(0x00000002) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConsumerMarkDeletePosition", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPulsarApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPulsarApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ConsumerMarkDeletePosition == nil { + m.ConsumerMarkDeletePosition = &MessageIdData{} + } + if err := m.ConsumerMarkDeletePosition.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -20981,10 +22193,7 @@ func (m *CommandGetTopicsOfNamespace) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -21095,10 +22304,7 @@ func (m *CommandGetTopicsOfNamespaceResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -21242,10 +22448,7 @@ func (m *CommandGetSchema) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -21447,10 +22650,7 @@ func (m *CommandGetSchemaResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -21597,10 +22797,7 @@ func (m *CommandGetOrCreateSchema) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -21769,10 +22966,7 @@ func (m *CommandGetOrCreateSchemaResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -21888,10 +23082,7 @@ func (m *CommandNewTxn) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -22060,10 +23251,7 @@ func (m *CommandNewTxnResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -22211,10 +23399,7 @@ func (m *CommandAddPartitionToTxn) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -22383,10 +23568,7 @@ func (m *CommandAddPartitionToTxnResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -22509,10 +23691,7 @@ func (m *Subscription) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -22665,10 +23844,7 @@ func (m *CommandAddSubscriptionToTxn) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -22837,10 +24013,7 @@ func (m *CommandAddSubscriptionToTxnResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -22976,10 +24149,7 @@ func (m *CommandEndTxn) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -23148,10 +24318,7 @@ func (m *CommandEndTxnResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -23314,16 +24481,33 @@ func (m *CommandEndTxnOnPartition) Unmarshal(dAtA []byte) error { } } m.TxnAction = &v + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TxnidLeastBitsOfLowWatermark", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TxnidLeastBitsOfLowWatermark = &v default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -23492,10 +24676,7 @@ func (m *CommandEndTxnOnPartitionResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -23661,16 +24842,33 @@ func (m *CommandEndTxnOnSubscription) Unmarshal(dAtA []byte) error { } } m.TxnAction = &v + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TxnidLeastBitsOfLowWatermark", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPulsarApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TxnidLeastBitsOfLowWatermark = &v default: iNdEx = preIndex skippy, err := skipPulsarApi(dAtA[iNdEx:]) if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -23839,10 +25037,7 @@ func (m *CommandEndTxnOnSubscriptionResponse) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { @@ -25754,10 +26949,7 @@ func (m *BaseCommand) Unmarshal(dAtA []byte) error { if err != nil { return err } - if skippy < 0 { - return ErrInvalidLengthPulsarApi - } - if (iNdEx + skippy) < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPulsarApi } if (iNdEx + skippy) > l { diff --git a/pulsar/internal/pulsar_proto/PulsarApi.proto b/pulsar/internal/pulsar_proto/PulsarApi.proto index bd5a6c13ed..b26024f8d1 100644 --- a/pulsar/internal/pulsar_proto/PulsarApi.proto +++ b/pulsar/internal/pulsar_proto/PulsarApi.proto @@ -40,6 +40,11 @@ message Schema { Time = 13; Timestamp = 14; KeyValue = 15; + Instant = 16; + LocalDate = 17; + LocalTime = 18; + LocalDateTime = 19; + ProtobufNative = 20; } required string name = 1; @@ -54,6 +59,8 @@ message MessageIdData { required uint64 entryId = 2; optional int32 partition = 3 [default = -1]; optional int32 batch_index = 4 [default = -1]; + repeated int64 ack_set = 5; + optional int32 batch_size = 6; } message KeyValue { @@ -85,6 +92,12 @@ enum CompressionType { SNAPPY = 4; } +enum ProducerAccessMode { + Shared = 0; // By default multiple producers can publish on a topic + Exclusive = 1; // Require exclusive access for producer. Fail immediately if there's already a producer connected. + WaitForExclusive = 2; // Producer creation is pending until it can acquire exclusive access +} + message MessageMetadata { required string producer_name = 1; required uint64 sequence_id = 2; @@ -130,11 +143,21 @@ message MessageMetadata { optional int32 marker_type = 20; // transaction related message info - optional uint64 txnid_least_bits = 22 [default = 0]; - optional uint64 txnid_most_bits = 23 [default = 0]; + optional uint64 txnid_least_bits = 22; + optional uint64 txnid_most_bits = 23; /// Add highest sequence id to support batch message with external sequence id optional uint64 highest_sequence_id = 24 [default = 0]; + + // Indicate if the message payload value is set + optional bool null_value = 25 [default = false]; + optional string uuid = 26; + optional int32 num_chunks_from_msg = 27; + optional int32 total_chunk_msg_size = 28; + optional int32 chunk_id = 29; + + // Indicate if the message partition key is set + optional bool null_partition_key = 30 [default = false]; } message SingleMessageMetadata { @@ -151,6 +174,16 @@ message SingleMessageMetadata { optional bytes ordering_key = 7; // Allows consumer retrieve the sequence id that the producer set. optional uint64 sequence_id = 8; + // Indicate if the message payload value is set + optional bool null_value = 9 [ default = false ]; + // Indicate if the message partition key is set + optional bool null_partition_key = 10 [ default = false]; +} + +// metadata added for entry from broker +message BrokerEntryMetadata { + optional uint64 broker_timestamp = 1; + optional uint64 index = 2; } enum ServerError { @@ -181,6 +214,16 @@ enum ServerError { TransactionCoordinatorNotFound = 20; // Transaction coordinator not found error InvalidTxnStatus = 21; // Invalid txn status error + NotAllowedError = 22; // Not allowed error + + TransactionConflict = 23; // Ack with transaction conflict + TransactionNotFound = 24; // Transaction not found + + ProducerFenced = 25; // When a producer asks and fail to get exclusive producer access, + // or loses the eclusive status after a reconnection, the broker will + // use this error to indicate that this producer is now permanently + // fenced. Applications are now supposed to close it and create a + // new producer } enum AuthMethod { @@ -211,6 +254,8 @@ enum ProtocolVersion { v14 = 14; // Add CommandAuthChallenge and CommandAuthResponse for mutual auth // Added Key_Shared subscription v15 = 15; // Add CommandGetOrCreateSchema and CommandGetOrCreateSchemaResponse + v16 = 16; // Add support for raw message metadata + v17 = 17; // Added support ack receipt } message CommandConnect { @@ -241,6 +286,7 @@ message CommandConnect { message FeatureFlags { optional bool supports_auth_refresh = 1 [default = false]; + optional bool supports_broker_entry_metadata = 2 [default = false]; } message CommandConnected { @@ -376,8 +422,7 @@ message CommandLookupTopic { // to the proxy. optional string original_auth_data = 5; optional string original_auth_method = 6; - - // net mode setting + // optional string advertised_listener_name = 7; } @@ -426,6 +471,16 @@ message CommandProducer { // Indicate the name of the producer is generated or user provided // Use default true here is in order to be forward compatible with the client optional bool user_provided_producer_name = 9 [default = true]; + + // Require that this producers will be the only producer allowed on the topic + optional ProducerAccessMode producer_access_mode = 10 [default = Shared]; + + // Topic epoch is used to fence off producers that reconnects after a new + // exclusive producer has already taken over. This id is assigned by the + // broker on the CommandProducerSuccess. The first time, the client will + // leave it empty and then it will always carry the same epoch number on + // the subsequent reconnections. + optional uint64 topic_epoch = 11; } message CommandSend { @@ -437,6 +492,7 @@ message CommandSend { /// Add highest sequence id to support batch message with external sequence id optional uint64 highest_sequence_id = 6 [default = 0]; + optional bool is_chunk =7 [default = false]; } message CommandSendReceipt { @@ -457,6 +513,7 @@ message CommandMessage { required uint64 consumer_id = 1; required MessageIdData message_id = 2; optional uint32 redelivery_count = 3 [default = 0]; + repeated int64 ack_set = 4; } message CommandAck { @@ -487,6 +544,7 @@ message CommandAck { optional uint64 txnid_least_bits = 6 [default = 0]; optional uint64 txnid_most_bits = 7 [default = 0]; + optional uint64 request_id = 8; } message CommandAckResponse { @@ -495,6 +553,7 @@ message CommandAckResponse { optional uint64 txnid_most_bits = 3 [default = 0]; optional ServerError error = 4; optional string message = 5; + optional uint64 request_id = 6; } // changes on active consumer @@ -561,6 +620,15 @@ message CommandProducerSuccess { // This will only be meaningful if deduplication has been enabled. optional int64 last_sequence_id = 3 [default = -1]; optional bytes schema_version = 4; + + // The topic epoch assigned by the broker. This field will only be set if we + // were requiring exclusive access when creating the producer. + optional uint64 topic_epoch = 5; + + // If producer is not "ready", the client will avoid to timeout the request + // for creating the producer. Instead it will wait indefinitely until it gets + // a subsequent `CommandProducerSuccess` with `producer_ready==true`. + optional bool producer_ready = 6 [default = true]; } message CommandError { @@ -634,6 +702,7 @@ message CommandGetLastMessageId { message CommandGetLastMessageIdResponse { required MessageIdData last_message_id = 1; required uint64 request_id = 2; + optional MessageIdData consumer_mark_delete_position = 3; } message CommandGetTopicsOfNamespace { @@ -758,6 +827,7 @@ message CommandEndTxnOnPartition { optional uint64 txnid_most_bits = 3 [default = 0]; optional string topic = 4; optional TxnAction txn_action = 5; + optional uint64 txnid_least_bits_of_low_watermark = 6; } message CommandEndTxnOnPartitionResponse { @@ -774,6 +844,7 @@ message CommandEndTxnOnSubscription { optional uint64 txnid_most_bits = 3 [default = 0]; optional Subscription subscription= 4; optional TxnAction txn_action = 5; + optional uint64 txnid_least_bits_of_low_watermark = 6; } message CommandEndTxnOnSubscriptionResponse { From 58ee24465bd19b4cf32ea52c0c3c113f74444029 Mon Sep 17 00:00:00 2001 From: yukun Date: Mon, 12 Jul 2021 15:53:44 +0800 Subject: [PATCH 22/25] [issue 516]Add lock for compressionProviders to fix data race problem (#533) * Add lock for compressionProviders to fix data race problem Signed-off-by: fishpenguin * Fix the code without releasing the lock Signed-off-by: fishpenguin * Change Mutex to RWMutex Signed-off-by: fishpenguin --- pulsar/consumer_partition.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index daaf759f92..cf9294978d 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -138,6 +138,7 @@ type partitionConsumer struct { log log.Logger + providersMutex sync.RWMutex compressionProviders map[pb.CompressionType]compression.Provider metrics *internal.TopicMetrics } @@ -850,9 +851,11 @@ func (pc *partitionConsumer) internalClose(req *closeRequest) { pc.log.Info("Closed consumer") } + pc.providersMutex.Lock() for _, provider := range pc.compressionProviders { provider.Close() } + pc.providersMutex.Unlock() pc.setConsumerState(consumerClosed) pc._getConn().DeleteConsumeHandler(pc.consumerID) @@ -1062,7 +1065,9 @@ func getPreviousMessage(mid trackingMessageID) trackingMessageID { } func (pc *partitionConsumer) Decompress(msgMeta *pb.MessageMetadata, payload internal.Buffer) (internal.Buffer, error) { + pc.providersMutex.RLock() provider, ok := pc.compressionProviders[msgMeta.GetCompression()] + pc.providersMutex.RUnlock() if !ok { var err error if provider, err = pc.initializeCompressionProvider(msgMeta.GetCompression()); err != nil { @@ -1070,7 +1075,9 @@ func (pc *partitionConsumer) Decompress(msgMeta *pb.MessageMetadata, payload int return nil, err } + pc.providersMutex.Lock() pc.compressionProviders[msgMeta.GetCompression()] = provider + pc.providersMutex.Unlock() } uncompressed, err := provider.Decompress(nil, payload.ReadableSlice(), int(msgMeta.GetUncompressedSize())) From f1939752ded912cc651e58e7b3c1bd12a2798724 Mon Sep 17 00:00:00 2001 From: xiaolong ran Date: Fri, 16 Jul 2021 17:58:10 +0800 Subject: [PATCH 23/25] Add send error logic for connection (#566) Fixes #564 Motivation Add command of SendError logic for connection Signed-off-by: xiaolongran --- pulsar/internal/commands.go | 2 + pulsar/internal/connection.go | 54 ++++++++++++++----- .../pulsartracing/producer_interceptor.go | 4 +- .../producer_interceptor_test.go | 3 +- 4 files changed, 48 insertions(+), 15 deletions(-) diff --git a/pulsar/internal/commands.go b/pulsar/internal/commands.go index 1adfba469a..af6bac5dd1 100644 --- a/pulsar/internal/commands.go +++ b/pulsar/internal/commands.go @@ -176,6 +176,8 @@ func baseCommand(cmdType pb.BaseCommand_Type, msg proto.Message) *pb.BaseCommand cmd.Pong = msg.(*pb.CommandPong) case pb.BaseCommand_SEND: cmd.Send = msg.(*pb.CommandSend) + case pb.BaseCommand_SEND_ERROR: + cmd.SendError = msg.(*pb.CommandSendError) case pb.BaseCommand_CLOSE_PRODUCER: cmd.CloseProducer = msg.(*pb.CommandCloseProducer) case pb.BaseCommand_CLOSE_CONSUMER: diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index a632816a16..9873ec8ced 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -535,6 +535,9 @@ func (c *connection) internalReceivedCommand(cmd *pb.BaseCommand, headersAndPayl case pb.BaseCommand_ERROR: c.handleResponseError(cmd.GetError()) + case pb.BaseCommand_SEND_ERROR: + c.handleSendError(cmd.GetError()) + case pb.BaseCommand_CLOSE_PRODUCER: c.handleCloseProducer(cmd.GetCloseProducer()) @@ -547,8 +550,6 @@ func (c *connection) internalReceivedCommand(cmd *pb.BaseCommand, headersAndPayl case pb.BaseCommand_SEND_RECEIPT: c.handleSendReceipt(cmd.GetSendReceipt()) - case pb.BaseCommand_SEND_ERROR: - case pb.BaseCommand_MESSAGE: c.handleMessage(cmd.GetMessage(), headersAndPayload) @@ -631,33 +632,25 @@ func (c *connection) internalSendRequest(req *request) { } func (c *connection) handleResponse(requestID uint64, response *pb.BaseCommand) { - c.pendingLock.Lock() - request, ok := c.pendingReqs[requestID] + request, ok := c.deletePendingRequest(requestID) if !ok { c.log.Warnf("Received unexpected response for request %d of type %s", requestID, response.Type) - c.pendingLock.Unlock() return } - delete(c.pendingReqs, requestID) - c.pendingLock.Unlock() request.callback(response, nil) } func (c *connection) handleResponseError(serverError *pb.CommandError) { requestID := serverError.GetRequestId() - c.pendingLock.Lock() - request, ok := c.pendingReqs[requestID] + + request, ok := c.deletePendingRequest(requestID) if !ok { c.log.Warnf("Received unexpected error response for request %d of type %s", requestID, serverError.GetError()) - c.pendingLock.Unlock() return } - delete(c.pendingReqs, requestID) - c.pendingLock.Unlock() - errMsg := fmt.Sprintf("server error: %s: %s", serverError.GetError(), serverError.GetMessage()) request.callback(nil, errors.New(errMsg)) } @@ -694,6 +687,16 @@ func (c *connection) handleMessage(response *pb.CommandMessage, payload Buffer) } } +func (c *connection) deletePendingRequest(requestID uint64) (*request, bool) { + c.pendingLock.Lock() + defer c.pendingLock.Unlock() + request, ok := c.pendingReqs[requestID] + if ok { + delete(c.pendingReqs, requestID) + } + return request, ok +} + func (c *connection) lastDataReceived() time.Time { c.lastDataReceivedLock.Lock() defer c.lastDataReceivedLock.Unlock() @@ -744,6 +747,31 @@ func (c *connection) handleAuthChallenge(authChallenge *pb.CommandAuthChallenge) c.writeCommand(baseCommand(pb.BaseCommand_AUTH_RESPONSE, cmdAuthResponse)) } +func (c *connection) handleSendError(cmdError *pb.CommandError) { + c.log.Warnf("Received send error from server: [%v] : [%s]", cmdError.GetError(), cmdError.GetMessage()) + + requestID := cmdError.GetRequestId() + + switch *cmdError.Error { + case pb.ServerError_NotAllowedError: + request, ok := c.deletePendingRequest(requestID) + if !ok { + c.log.Warnf("Received unexpected error response for request %d of type %s", + requestID, cmdError.GetError()) + return + } + + errMsg := fmt.Sprintf("server error: %s: %s", cmdError.GetError(), cmdError.GetMessage()) + request.callback(nil, errors.New(errMsg)) + case pb.ServerError_TopicTerminatedError: + // TODO: no-op + default: + // By default, for transient error, let the reconnection logic + // to take place and re-establish the produce again + c.Close() + } +} + func (c *connection) handleCloseConsumer(closeConsumer *pb.CommandCloseConsumer) { consumerID := closeConsumer.GetConsumerId() c.log.Infof("Broker notification of Closed consumer: %d", consumerID) diff --git a/pulsar/internal/pulsartracing/producer_interceptor.go b/pulsar/internal/pulsartracing/producer_interceptor.go index b400e57785..6c7728cf0a 100644 --- a/pulsar/internal/pulsartracing/producer_interceptor.go +++ b/pulsar/internal/pulsartracing/producer_interceptor.go @@ -33,7 +33,9 @@ func (t *ProducerInterceptor) BeforeSend(producer pulsar.Producer, message *puls buildAndInjectSpan(message, producer).Finish() } -func (t *ProducerInterceptor) OnSendAcknowledgement(producer pulsar.Producer, message *pulsar.ProducerMessage, msgID pulsar.MessageID) { +func (t *ProducerInterceptor) OnSendAcknowledgement(producer pulsar.Producer, + message *pulsar.ProducerMessage, + msgID pulsar.MessageID) { } func buildAndInjectSpan(message *pulsar.ProducerMessage, producer pulsar.Producer) opentracing.Span { diff --git a/pulsar/internal/pulsartracing/producer_interceptor_test.go b/pulsar/internal/pulsartracing/producer_interceptor_test.go index b146e4e93b..8d8e6965b8 100644 --- a/pulsar/internal/pulsartracing/producer_interceptor_test.go +++ b/pulsar/internal/pulsartracing/producer_interceptor_test.go @@ -55,7 +55,8 @@ func (p *mockProducer) Send(context.Context, *pulsar.ProducerMessage) (pulsar.Me return nil, nil } -func (p *mockProducer) SendAsync(context.Context, *pulsar.ProducerMessage, func(pulsar.MessageID, *pulsar.ProducerMessage, error)) { +func (p *mockProducer) SendAsync(context.Context, *pulsar.ProducerMessage, + func(pulsar.MessageID, *pulsar.ProducerMessage, error)) { } func (p *mockProducer) LastSequenceID() int64 { From 179060fa145ec52f58e67202942b05b271727b65 Mon Sep 17 00:00:00 2001 From: jiangbo9510 Date: Mon, 19 Jul 2021 11:54:31 +0800 Subject: [PATCH 24/25] [Issue 527] fix send goroutine blocked (#530) * fix failTimeoutMessages() poll wrong item from block queue * add leave out lock Co-authored-by: boojiang Co-authored-by: xiaolongran --- pulsar/internal/blocking_queue.go | 17 +++++++++++++++++ pulsar/producer_partition.go | 30 +++++++++++++++++++++++------- 2 files changed, 40 insertions(+), 7 deletions(-) diff --git a/pulsar/internal/blocking_queue.go b/pulsar/internal/blocking_queue.go index 8162301147..b44ec1668c 100644 --- a/pulsar/internal/blocking_queue.go +++ b/pulsar/internal/blocking_queue.go @@ -32,6 +32,9 @@ type BlockingQueue interface { // Poll dequeue one item, return nil if queue is empty Poll() interface{} + // CompareAndPoll compare the first item and poll it if meet the conditions + CompareAndPoll(compare func(item interface{}) bool) interface{} + // Peek return the first item without dequeing, return nil if queue is empty Peek() interface{} @@ -117,6 +120,20 @@ func (bq *blockingQueue) Poll() interface{} { return bq.dequeue() } +func (bq *blockingQueue) CompareAndPoll(compare func(interface{}) bool) interface{} { + bq.mutex.Lock() + defer bq.mutex.Unlock() + + if bq.size == 0 { + return nil + } + + if compare(bq.items[bq.headIdx]) { + return bq.dequeue() + } + return nil +} + func (bq *blockingQueue) Peek() interface{} { bq.mutex.Lock() defer bq.mutex.Unlock() diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 002d261cfd..8b3d33dadd 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -515,21 +515,37 @@ func (p *partitionProducer) failTimeoutMessages() { // iterate at most viewSize items for i := 0; i < viewSize; i++ { - item := p.pendingQueue.Poll() + tickerNeedWaiting := time.Duration(0) + item := p.pendingQueue.CompareAndPoll( + func(m interface{}) bool { + if m == nil { + return false + } + + pi := m.(*pendingItem) + pi.Lock() + defer pi.Unlock() + if nextWaiting := diff(pi.sentAt); nextWaiting > 0 { + // current and subsequent items not timeout yet, stop iterating + tickerNeedWaiting = nextWaiting + return false + } + return true + }) + if item == nil { t.Reset(p.options.SendTimeout) break } - pi := item.(*pendingItem) - pi.Lock() - if nextWaiting := diff(pi.sentAt); nextWaiting > 0 { - // current and subsequent items not timeout yet, stop iterating - t.Reset(nextWaiting) - pi.Unlock() + if tickerNeedWaiting > 0 { + t.Reset(tickerNeedWaiting) break } + pi := item.(*pendingItem) + pi.Lock() + for _, i := range pi.sendRequests { sr := i.(*sendRequest) if sr.msg != nil { From 5e88b015e9aa2c3fd993208c35e47cf1852be07d Mon Sep 17 00:00:00 2001 From: xiaolong ran Date: Mon, 19 Jul 2021 17:44:28 +0800 Subject: [PATCH 25/25] Add license file for depend libs (#567) Signed-off-by: xiaolongran ## Motivation Add license file for depend libs --- distribution/LICENSE.bin.txt | 12 ++ distribution/license/LICENSE-athenz.txt | 202 +++++++++++++++++++ distribution/license/LICENSE-go-avro.txt | 201 ++++++++++++++++++ distribution/license/LICENSE-go-spew.txt | 15 ++ distribution/license/LICENSE-oauth2.txt | 27 +++ distribution/license/LICENSE-opentracing.txt | 202 +++++++++++++++++++ distribution/license/LICENSE-pretty.txt | 19 ++ distribution/license/LICENSE-prometheus.txt | 201 ++++++++++++++++++ distribution/license/LICENSE-uuid.txt | 27 +++ distribution/license/LICENSE-zstd.txt | 27 +++ 10 files changed, 933 insertions(+) create mode 100644 distribution/license/LICENSE-athenz.txt create mode 100644 distribution/license/LICENSE-go-avro.txt create mode 100644 distribution/license/LICENSE-go-spew.txt create mode 100644 distribution/license/LICENSE-oauth2.txt create mode 100644 distribution/license/LICENSE-opentracing.txt create mode 100644 distribution/license/LICENSE-pretty.txt create mode 100644 distribution/license/LICENSE-prometheus.txt create mode 100644 distribution/license/LICENSE-uuid.txt create mode 100644 distribution/license/LICENSE-zstd.txt diff --git a/distribution/LICENSE.bin.txt b/distribution/LICENSE.bin.txt index 07c331b463..e37200faac 100644 --- a/distribution/LICENSE.bin.txt +++ b/distribution/LICENSE.bin.txt @@ -207,20 +207,32 @@ This projects includes binary packages with the following licenses: The Apache Software License, Version 2.0 * github.com/spf13/cobra v0.0.3 -- distribution/license/LICENSE-cobra.txt + * github.com/AthenZ/athenz v1.10.15 -- distribution/license/LICENSE-athenz.txt * github.com/inconshreveable/mousetrap v1.0.0 -- distribution/license/LICENSE-mousetrap.txt + * github.com/linkedin/goavro/v2 v2.9.8 -- distribution/license/LICENSE-go-avro.txt + * github.com/opentracing/opentracing-go v1.2.0 -- distribution/license/LICENSE-opentracing.txt + * github.com/prometheus/client_golang v1.7.1 -- distribution/license/LICENSE-prometheus.txt MIT License * github.com/stretchr/testify v1.3.0 -- distribution/license/LICENSE-testify.txt + * github.com/kr/pretty v0.2.0 -- distribution/license/LICENSE-pretty.txt * github.com/sirupsen/logrus v1.4.1 -- distribution/license/LICENSE-logrus.txt GNU GENERAL PUBLIC LICENSE Version 3 * github.com/beefsack/go-rate v0.0.0-20180408011153-efa7637bb9b6 -- distribution/license/LICENSE-go-rate.txt BSD License + * github.com/DataDog/zstd v1.4.6-0.20210211175136-c6db21d202f4 -- distribution/license/LICENSE-zstd.txt * github.com/klauspost/compress v1.9.2 -- distribution/license/LICENSE-compress.txt + * github.com/google/uuid v1.1.2 -- distribution/license/LICENSE-uuid.txt * github.com/pkg/errors v0.8.1 -- distribution/license/LICENSE-errors.txt * github.com/google/go-cmp v0.3.0 -- distribution/license/LICENSE-go-cmp.txt * github.com/pierrec/lz4 v2.0.5+incompatible -- distribution/license/LICENSE-lz4.txt * github.com/golang/protobuf v1.3.1 -- distribution/license/LICENSE-protobuf.txt * github.com/spf13/pflag v1.0.3 -- distribution/license/LICENSE-pflag.txt * github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b -- distribution/license/LICENSE-perks.txt + * github.com/spaolacci/murmur3 v1.1.0 -- distribution/license/LICENSE-murmur3.txt + * golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d -- distribution/license/LICENSE-oauth2.txt + +ISC License + * github.com/davecgh/go-spew v1.1.1 -- distribution/license/LICENSE-go-spew.txt diff --git a/distribution/license/LICENSE-athenz.txt b/distribution/license/LICENSE-athenz.txt new file mode 100644 index 0000000000..d645695673 --- /dev/null +++ b/distribution/license/LICENSE-athenz.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/distribution/license/LICENSE-go-avro.txt b/distribution/license/LICENSE-go-avro.txt new file mode 100644 index 0000000000..261eeb9e9f --- /dev/null +++ b/distribution/license/LICENSE-go-avro.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/distribution/license/LICENSE-go-spew.txt b/distribution/license/LICENSE-go-spew.txt new file mode 100644 index 0000000000..7304bdf8cb --- /dev/null +++ b/distribution/license/LICENSE-go-spew.txt @@ -0,0 +1,15 @@ +ISC License + +Copyright (c) 2012-2016 Dave Collins + +Permission to use, copy, modify, and/or distribute this software for any +purpose with or without fee is hereby granted, provided that the above +copyright notice and this permission notice appear in all copies. + +THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES +WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR +ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES +WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN +ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF +OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. \ No newline at end of file diff --git a/distribution/license/LICENSE-oauth2.txt b/distribution/license/LICENSE-oauth2.txt new file mode 100644 index 0000000000..ea5ea89869 --- /dev/null +++ b/distribution/license/LICENSE-oauth2.txt @@ -0,0 +1,27 @@ +Copyright (c) 2009 The Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/distribution/license/LICENSE-opentracing.txt b/distribution/license/LICENSE-opentracing.txt new file mode 100644 index 0000000000..d645695673 --- /dev/null +++ b/distribution/license/LICENSE-opentracing.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/distribution/license/LICENSE-pretty.txt b/distribution/license/LICENSE-pretty.txt new file mode 100644 index 0000000000..2e9e336a3f --- /dev/null +++ b/distribution/license/LICENSE-pretty.txt @@ -0,0 +1,19 @@ +Copyright 2012 Keith Rarick + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file diff --git a/distribution/license/LICENSE-prometheus.txt b/distribution/license/LICENSE-prometheus.txt new file mode 100644 index 0000000000..261eeb9e9f --- /dev/null +++ b/distribution/license/LICENSE-prometheus.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/distribution/license/LICENSE-uuid.txt b/distribution/license/LICENSE-uuid.txt new file mode 100644 index 0000000000..0c2c410a76 --- /dev/null +++ b/distribution/license/LICENSE-uuid.txt @@ -0,0 +1,27 @@ +Copyright (c) 2009,2014 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/distribution/license/LICENSE-zstd.txt b/distribution/license/LICENSE-zstd.txt new file mode 100644 index 0000000000..345c1eb932 --- /dev/null +++ b/distribution/license/LICENSE-zstd.txt @@ -0,0 +1,27 @@ +Simplified BSD License + +Copyright (c) 2016, Datadog +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + * Neither the name of the copyright holder nor the names of its contributors + may be used to endorse or promote products derived from this software + without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.