From d92fb1407d3d39c8a498dd7c7abdc0bbb3fc7e1a Mon Sep 17 00:00:00 2001 From: Nitin Goyal Date: Sun, 18 Dec 2022 15:21:16 +0530 Subject: [PATCH 001/348] Default retry and dlq topic name fixed as per the doc (#891) * default retry and dlq name fixed based on doc default retry and dlq name fixed based on doc * backword compatibility fixed Signed-off-by: Nitin Goyal * testcase fixed Signed-off-by: Nitin Goyal * testcase fixed Signed-off-by: Nitin Goyal * testcase fixed Signed-off-by: Nitin Goyal * Lint fixed * Fix Retry topic when topic is partitioned * Fix Retry topic when topic is partitioned * RETRY and DLQ topic name bug fixed for partition topics * RETRY and DLQ topic name bug fixed for partition topics * Bug fix Signed-off-by: Nitin Goyal --- pulsar/consumer_impl.go | 22 ++++++++++++++++++++-- pulsar/consumer_test.go | 6 +++--- 2 files changed, 23 insertions(+), 5 deletions(-) diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index 707c13c9c6..e5290a1db0 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -122,8 +122,26 @@ func newConsumer(client *client, options ConsumerOptions) (Consumer, error) { return nil, err } - retryTopic := tn.Domain + "://" + tn.Namespace + "/" + options.SubscriptionName + RetryTopicSuffix - dlqTopic := tn.Domain + "://" + tn.Namespace + "/" + options.SubscriptionName + DlqTopicSuffix + topicName := internal.TopicNameWithoutPartitionPart(tn) + + retryTopic := topicName + "-" + options.SubscriptionName + RetryTopicSuffix + dlqTopic := topicName + "-" + options.SubscriptionName + DlqTopicSuffix + + oldRetryTopic := tn.Domain + "://" + tn.Namespace + "/" + options.SubscriptionName + RetryTopicSuffix + oldDlqTopic := tn.Domain + "://" + tn.Namespace + "/" + options.SubscriptionName + DlqTopicSuffix + + if r, err := client.lookupService.GetPartitionedTopicMetadata(oldRetryTopic); err == nil && + r != nil && + r.Partitions > 0 { + retryTopic = oldRetryTopic + } + + if r, err := client.lookupService.GetPartitionedTopicMetadata(oldDlqTopic); err == nil && + r != nil && + r.Partitions > 0 { + dlqTopic = oldDlqTopic + } + if options.DLQ == nil { options.DLQ = &DLQPolicy{ MaxDeliveries: MaxReconsumeTimes, diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 7a7f118579..355e33b302 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -1498,7 +1498,7 @@ func TestRLQ(t *testing.T) { // 3. Create consumer on the DLQ topic to verify the routing dlqConsumer, err := client.Subscribe(ConsumerOptions{ - Topic: "persistent://public/default/" + subName + "-DLQ", + Topic: "persistent://public/default/" + topic + "-" + subName + "-DLQ", SubscriptionName: subName, SubscriptionInitialPosition: SubscriptionPositionEarliest, }) @@ -1603,7 +1603,7 @@ func TestRLQMultiTopics(t *testing.T) { // subscribe DLQ Topic dlqConsumer, err := client.Subscribe(ConsumerOptions{ - Topic: subName + "-DLQ", + Topic: topics[0] + "-" + subName + "-DLQ", SubscriptionName: subName, SubscriptionInitialPosition: SubscriptionPositionEarliest, }) @@ -1710,7 +1710,7 @@ func TestRLQSpecifiedPartitionTopic(t *testing.T) { // subscribe DLQ Topic dlqConsumer, err := client.Subscribe(ConsumerOptions{ - Topic: subName + "-DLQ", + Topic: normalTopic + "-" + subName + "-DLQ", SubscriptionName: subName, SubscriptionInitialPosition: SubscriptionPositionEarliest, }) From 1d3499a18d526b4b1aef0bdbbc54ac812b8ae0c0 Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Fri, 23 Dec 2022 15:13:04 +0800 Subject: [PATCH 002/348] fix: fix 923 (#924) Remove the outdated interface description of SeekByTime. More details here #923. --- pulsar/consumer.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/pulsar/consumer.go b/pulsar/consumer.go index 0a515e0121..0e89fc0cb3 100644 --- a/pulsar/consumer.go +++ b/pulsar/consumer.go @@ -267,9 +267,6 @@ type Consumer interface { // SeekByTime resets the subscription associated with this consumer to a specific message publish time. // - // Note: this operation can only be done on non-partitioned topics. For these, one can rather perform the seek() on - // the individual partitions. - // // @param time // the message publish time when to reposition the subscription // From cb983a7ebe1390b75832c58c15acb28796c35983 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 3 Jan 2023 12:07:01 +0800 Subject: [PATCH 003/348] Move out the auth package from internal (#914) Signed-off-by: Zixuan Liu --- pulsar/{internal => }/auth/athenz.go | 0 pulsar/{internal => }/auth/athenz_test.go | 4 ++-- pulsar/{internal => }/auth/basic.go | 0 pulsar/{internal => }/auth/basic_test.go | 0 pulsar/{internal => }/auth/disabled.go | 0 pulsar/{internal => }/auth/oauth2.go | 0 pulsar/{internal => }/auth/oauth2_test.go | 0 pulsar/{internal => }/auth/provider.go | 0 pulsar/{internal => }/auth/tls.go | 0 pulsar/{internal => }/auth/token.go | 0 pulsar/client.go | 2 +- pulsar/client_impl.go | 7 +++---- pulsar/client_impl_test.go | 6 ++---- pulsar/internal/connection.go | 3 ++- pulsar/internal/connection_pool.go | 3 ++- pulsar/internal/http_client.go | 2 +- 16 files changed, 13 insertions(+), 14 deletions(-) rename pulsar/{internal => }/auth/athenz.go (100%) rename pulsar/{internal => }/auth/athenz_test.go (96%) rename pulsar/{internal => }/auth/basic.go (100%) rename pulsar/{internal => }/auth/basic_test.go (100%) rename pulsar/{internal => }/auth/disabled.go (100%) rename pulsar/{internal => }/auth/oauth2.go (100%) rename pulsar/{internal => }/auth/oauth2_test.go (100%) rename pulsar/{internal => }/auth/provider.go (100%) rename pulsar/{internal => }/auth/tls.go (100%) rename pulsar/{internal => }/auth/token.go (100%) diff --git a/pulsar/internal/auth/athenz.go b/pulsar/auth/athenz.go similarity index 100% rename from pulsar/internal/auth/athenz.go rename to pulsar/auth/athenz.go diff --git a/pulsar/internal/auth/athenz_test.go b/pulsar/auth/athenz_test.go similarity index 96% rename from pulsar/internal/auth/athenz_test.go rename to pulsar/auth/athenz_test.go index 2556e93ee6..97cc0aee19 100644 --- a/pulsar/internal/auth/athenz_test.go +++ b/pulsar/auth/athenz_test.go @@ -27,10 +27,10 @@ import ( zms "github.com/AthenZ/athenz/libs/go/zmssvctoken" zts "github.com/AthenZ/athenz/libs/go/ztsroletoken" "github.com/stretchr/testify/assert" - mock "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/mock" ) -const tlsClientKeyPath = "../../../integration-tests/certs/client-key.pem" +const tlsClientKeyPath = "../../integration-tests/certs/client-key.pem" type MockTokenBuilder struct { mock.Mock diff --git a/pulsar/internal/auth/basic.go b/pulsar/auth/basic.go similarity index 100% rename from pulsar/internal/auth/basic.go rename to pulsar/auth/basic.go diff --git a/pulsar/internal/auth/basic_test.go b/pulsar/auth/basic_test.go similarity index 100% rename from pulsar/internal/auth/basic_test.go rename to pulsar/auth/basic_test.go diff --git a/pulsar/internal/auth/disabled.go b/pulsar/auth/disabled.go similarity index 100% rename from pulsar/internal/auth/disabled.go rename to pulsar/auth/disabled.go diff --git a/pulsar/internal/auth/oauth2.go b/pulsar/auth/oauth2.go similarity index 100% rename from pulsar/internal/auth/oauth2.go rename to pulsar/auth/oauth2.go diff --git a/pulsar/internal/auth/oauth2_test.go b/pulsar/auth/oauth2_test.go similarity index 100% rename from pulsar/internal/auth/oauth2_test.go rename to pulsar/auth/oauth2_test.go diff --git a/pulsar/internal/auth/provider.go b/pulsar/auth/provider.go similarity index 100% rename from pulsar/internal/auth/provider.go rename to pulsar/auth/provider.go diff --git a/pulsar/internal/auth/tls.go b/pulsar/auth/tls.go similarity index 100% rename from pulsar/internal/auth/tls.go rename to pulsar/auth/tls.go diff --git a/pulsar/internal/auth/token.go b/pulsar/auth/token.go similarity index 100% rename from pulsar/internal/auth/token.go rename to pulsar/auth/token.go diff --git a/pulsar/client.go b/pulsar/client.go index 29a582fb0a..d75c977b40 100644 --- a/pulsar/client.go +++ b/pulsar/client.go @@ -21,7 +21,7 @@ import ( "crypto/tls" "time" - "github.com/apache/pulsar-client-go/pulsar/internal/auth" + "github.com/apache/pulsar-client-go/pulsar/auth" "github.com/apache/pulsar-client-go/pulsar/log" "github.com/prometheus/client_golang/prometheus" ) diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index 7cac0977b9..e794e3d71e 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -22,12 +22,11 @@ import ( "net/url" "time" - "github.com/prometheus/client_golang/prometheus" - "github.com/sirupsen/logrus" - + "github.com/apache/pulsar-client-go/pulsar/auth" "github.com/apache/pulsar-client-go/pulsar/internal" - "github.com/apache/pulsar-client-go/pulsar/internal/auth" "github.com/apache/pulsar-client-go/pulsar/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/sirupsen/logrus" ) const ( diff --git a/pulsar/client_impl_test.go b/pulsar/client_impl_test.go index c98a5428bb..6ead0575bd 100644 --- a/pulsar/client_impl_test.go +++ b/pulsar/client_impl_test.go @@ -28,12 +28,10 @@ import ( "testing" "time" - "github.com/stretchr/testify/require" - + "github.com/apache/pulsar-client-go/pulsar/auth" "github.com/apache/pulsar-client-go/pulsar/internal" - - "github.com/apache/pulsar-client-go/pulsar/internal/auth" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestClient(t *testing.T) { diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 00a9521de6..be95240b36 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -29,9 +29,10 @@ import ( "sync/atomic" "time" + "github.com/apache/pulsar-client-go/pulsar/auth" + "google.golang.org/protobuf/proto" - "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" diff --git a/pulsar/internal/connection_pool.go b/pulsar/internal/connection_pool.go index cb172a12aa..4f6d6566fe 100644 --- a/pulsar/internal/connection_pool.go +++ b/pulsar/internal/connection_pool.go @@ -24,7 +24,8 @@ import ( "sync/atomic" "time" - "github.com/apache/pulsar-client-go/pulsar/internal/auth" + "github.com/apache/pulsar-client-go/pulsar/auth" + "github.com/apache/pulsar-client-go/pulsar/log" ) diff --git a/pulsar/internal/http_client.go b/pulsar/internal/http_client.go index 7abf70d4b6..672a65ed69 100644 --- a/pulsar/internal/http_client.go +++ b/pulsar/internal/http_client.go @@ -29,7 +29,7 @@ import ( "path" "time" - "github.com/apache/pulsar-client-go/pulsar/internal/auth" + "github.com/apache/pulsar-client-go/pulsar/auth" "github.com/apache/pulsar-client-go/pulsar/log" From 23777362f3d132f98eff56c4f511f761227d8cf4 Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Tue, 3 Jan 2023 15:56:03 +0800 Subject: [PATCH 004/348] [Issue 183] [Feat] Support cumulative acknowledge (#903) Master Issue: #183 ### Motivation Cumulative acknowledgement is a useful feature. Users can use this feature to ack messages in the stream up to (and including) provided message. Issue #183 shows more details. ### Modifications - Add two api `AckCumulative` and `AckIDCumulative` for `Consumer`. ``` golang // AckCumulative the reception of all the messages in the stream up to (and including) // the provided message. AckCumulative(msg Message) error // AckIDCumulative the reception of all the messages in the stream up to (and including) // the provided message, identified by its MessageID AckIDCumulative(msgID MessageID) error ``` - Add the `AckCumulative` and `AckIDCumulative` implementation for `consumer`, `multiTopicConsumer`, `regexConsumer` and `mockConsumer`. - Add the unit test `TestConsumerNoBatchCumulativeAck` `TestConsumerBatchCumulativeAck` `TestCumulativeAckWithResponse` for cumulative ack in `consumer_test.go`. --- pulsar/consumer.go | 8 + pulsar/consumer_impl.go | 22 ++ pulsar/consumer_multitopic.go | 27 +++ pulsar/consumer_partition.go | 78 +++++- pulsar/consumer_regex.go | 27 +++ pulsar/consumer_test.go | 224 ++++++++++++++++++ pulsar/impl_message.go | 44 +++- pulsar/impl_message_test.go | 17 ++ .../consumer_interceptor_test.go | 8 + 9 files changed, 449 insertions(+), 6 deletions(-) diff --git a/pulsar/consumer.go b/pulsar/consumer.go index 0e89fc0cb3..49a24d56d6 100644 --- a/pulsar/consumer.go +++ b/pulsar/consumer.go @@ -234,6 +234,14 @@ type Consumer interface { // AckID the consumption of a single message, identified by its MessageID AckID(MessageID) error + // AckCumulative the reception of all the messages in the stream up to (and including) + // the provided message. + AckCumulative(msg Message) error + + // AckIDCumulative the reception of all the messages in the stream up to (and including) + // the provided message, identified by its MessageID + AckIDCumulative(msgID MessageID) error + // ReconsumeLater mark a message for redelivery after custom delay ReconsumeLater(msg Message, delay time.Duration) diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index e5290a1db0..11a3cf70af 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -38,6 +38,8 @@ type acker interface { // AckID does not handle errors returned by the Broker side, so no need to wait for doneCh to finish. AckID(id MessageID) error AckIDWithResponse(id MessageID) error + AckIDCumulative(msgID MessageID) error + AckIDWithResponseCumulative(msgID MessageID) error NackID(id MessageID) NackMsg(msg Message) } @@ -496,6 +498,26 @@ func (c *consumer) AckID(msgID MessageID) error { return c.consumers[msgID.PartitionIdx()].AckID(msgID) } +// AckCumulative the reception of all the messages in the stream up to (and including) +// the provided message, identified by its MessageID +func (c *consumer) AckCumulative(msg Message) error { + return c.AckIDCumulative(msg.ID()) +} + +// AckIDCumulative the reception of all the messages in the stream up to (and including) +// the provided message, identified by its MessageID +func (c *consumer) AckIDCumulative(msgID MessageID) error { + if err := c.checkMsgIDPartition(msgID); err != nil { + return err + } + + if c.options.AckWithResponse { + return c.consumers[msgID.PartitionIdx()].AckIDWithResponseCumulative(msgID) + } + + return c.consumers[msgID.PartitionIdx()].AckIDCumulative(msgID) +} + // ReconsumeLater mark a message for redelivery after custom delay func (c *consumer) ReconsumeLater(msg Message, delay time.Duration) { if delay < 0 { diff --git a/pulsar/consumer_multitopic.go b/pulsar/consumer_multitopic.go index d32ce31c50..e653926322 100644 --- a/pulsar/consumer_multitopic.go +++ b/pulsar/consumer_multitopic.go @@ -143,6 +143,33 @@ func (c *multiTopicConsumer) AckID(msgID MessageID) error { return mid.consumer.AckID(msgID) } +// AckCumulative the reception of all the messages in the stream up to (and including) +// the provided message +func (c *multiTopicConsumer) AckCumulative(msg Message) error { + return c.AckIDCumulative(msg.ID()) +} + +// AckIDCumulative the reception of all the messages in the stream up to (and including) +// the provided message, identified by its MessageID +func (c *multiTopicConsumer) AckIDCumulative(msgID MessageID) error { + mid, ok := toTrackingMessageID(msgID) + if !ok { + c.log.Warnf("invalid message id type %T", msgID) + return errors.New("invalid message id type in multi_consumer") + } + + if mid.consumer == nil { + c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID) + return errors.New("unable to ack message because consumer is nil") + } + + if c.options.AckWithResponse { + return mid.consumer.AckIDWithResponseCumulative(msgID) + } + + return mid.consumer.AckIDCumulative(msgID) +} + func (c *multiTopicConsumer) ReconsumeLater(msg Message, delay time.Duration) { names, err := validateTopicNames(msg.Topic()) if err != nil { diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 4215b69363..7f5b6df711 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -451,6 +451,7 @@ func (pc *partitionConsumer) AckIDWithResponse(msgID MessageID) error { ackReq := new(ackRequest) ackReq.doneCh = make(chan struct{}) + ackReq.ackType = individualAck if !trackingID.Undefined() && trackingID.ack() { pc.metrics.AcksCounter.Inc() pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9) @@ -483,6 +484,7 @@ func (pc *partitionConsumer) AckID(msgID MessageID) error { ackReq := new(ackRequest) ackReq.doneCh = make(chan struct{}) + ackReq.ackType = individualAck if !trackingID.Undefined() && trackingID.ack() { pc.metrics.AcksCounter.Inc() pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9) @@ -497,6 +499,62 @@ func (pc *partitionConsumer) AckID(msgID MessageID) error { return nil } +func (pc *partitionConsumer) AckIDCumulative(msgID MessageID) error { + return pc.internalAckIDCumulative(msgID, false) +} + +func (pc *partitionConsumer) AckIDWithResponseCumulative(msgID MessageID) error { + return pc.internalAckIDCumulative(msgID, true) +} + +func (pc *partitionConsumer) internalAckIDCumulative(msgID MessageID, withResponse bool) error { + if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing { + pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer") + return errors.New("consumer state is closed") + } + + // chunk message id will be converted to tracking message id + trackingID, ok := toTrackingMessageID(msgID) + if !ok { + return errors.New("failed to convert trackingMessageID") + } + if trackingID.Undefined() { + return nil + } + + ackReq := new(ackRequest) + ackReq.doneCh = make(chan struct{}) + ackReq.ackType = cumulativeAck + if trackingID.ackCumulative() { + ackReq.msgID = trackingID + } else if !trackingID.tracker.hasPrevBatchAcked() { + // get previous batch message id + ackReq.msgID = trackingID.prev() + trackingID.tracker.setPrevBatchAcked() + } else { + // waiting for all the msgs are acked in this batch + return nil + } + + pc.metrics.AcksCounter.Inc() + pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9) + // send ack request to eventsCh + pc.eventsCh <- ackReq + + if withResponse { + // wait for the request to complete if withResponse set true + <-ackReq.doneCh + } + + pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID) + + if cmid, ok := toChunkedMessageID(msgID); ok { + pc.unAckChunksTracker.remove(cmid) + } + + return nil +} + func (pc *partitionConsumer) NackID(msgID MessageID) { if cmid, ok := toChunkedMessageID(msgID); ok { pc.unAckChunksTracker.nack(cmid) @@ -711,7 +769,13 @@ func (pc *partitionConsumer) internalAck(req *ackRequest) { cmdAck := &pb.CommandAck{ ConsumerId: proto.Uint64(pc.consumerID), MessageId: messageIDs, - AckType: pb.CommandAck_Individual.Enum(), + } + + switch req.ackType { + case individualAck: + cmdAck.AckType = pb.CommandAck_Individual.Enum() + case cumulativeAck: + cmdAck.AckType = pb.CommandAck_Cumulative.Enum() } if pc.options.ackWithResponse { @@ -1189,10 +1253,16 @@ func (pc *partitionConsumer) dispatcher() { } } +const ( + individualAck = iota + cumulativeAck +) + type ackRequest struct { - doneCh chan struct{} - msgID trackingMessageID - err error + doneCh chan struct{} + msgID trackingMessageID + ackType int + err error } type unsubscribeRequest struct { diff --git a/pulsar/consumer_regex.go b/pulsar/consumer_regex.go index 55f3d7a37b..36d2245f26 100644 --- a/pulsar/consumer_regex.go +++ b/pulsar/consumer_regex.go @@ -187,6 +187,33 @@ func (c *regexConsumer) AckID(msgID MessageID) error { return mid.consumer.AckID(msgID) } +// AckCumulative the reception of all the messages in the stream up to (and including) +// the provided message. +func (c *regexConsumer) AckCumulative(msg Message) error { + return c.AckIDCumulative(msg.ID()) +} + +// AckIDCumulative the reception of all the messages in the stream up to (and including) +// the provided message, identified by its MessageID +func (c *regexConsumer) AckIDCumulative(msgID MessageID) error { + mid, ok := toTrackingMessageID(msgID) + if !ok { + c.log.Warnf("invalid message id type %T", msgID) + return errors.New("invalid message id type") + } + + if mid.consumer == nil { + c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID) + return errors.New("unable to ack message because consumer is nil") + } + + if c.options.AckWithResponse { + return mid.consumer.AckIDWithResponseCumulative(msgID) + } + + return mid.consumer.AckIDCumulative(msgID) +} + func (c *regexConsumer) Nack(msg Message) { if c.options.EnableDefaultNackBackoffPolicy || c.options.NackBackoffPolicy != nil { msgID := msg.ID() diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 355e33b302..891fcd23c1 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -867,6 +867,193 @@ func TestConsumerAck(t *testing.T) { } } +func TestConsumerNoBatchCumulativeAck(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + + topicName := newTopicName() + ctx := context.Background() + + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topicName, + // disable batching + DisableBatching: true, + }) + assert.Nil(t, err) + defer producer.Close() + + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topicName, + SubscriptionName: "sub-1", + Type: Exclusive, + }) + assert.Nil(t, err) + + const N = 100 + + for i := 0; i < N; i++ { + if _, err := producer.Send(ctx, &ProducerMessage{ + Payload: []byte(fmt.Sprintf("msg-content-%d", i)), + }); err != nil { + t.Fatal(err) + } + } + + for i := 0; i < N; i++ { + msg, err := consumer.Receive(ctx) + assert.Nil(t, err) + assert.Equal(t, fmt.Sprintf("msg-content-%d", i), string(msg.Payload())) + + if i == N/2-1 { + // cumulative acks the first half of messages + consumer.AckCumulative(msg) + } + } + + consumer.Close() + + // Subscribe again + consumer, err = client.Subscribe(ConsumerOptions{ + Topic: topicName, + SubscriptionName: "sub-1", + Type: Exclusive, + }) + assert.Nil(t, err) + defer consumer.Close() + + // We should only receive the 2nd half of messages + for i := N / 2; i < N; i++ { + msg, err := consumer.Receive(ctx) + assert.Nil(t, err) + assert.Equal(t, fmt.Sprintf("msg-content-%d", i), string(msg.Payload())) + + consumer.Ack(msg) + } +} + +func TestConsumerBatchCumulativeAck(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + + topicName := newTopicName() + ctx := context.Background() + + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topicName, + }) + assert.Nil(t, err) + defer producer.Close() + + c1, err := client.Subscribe(ConsumerOptions{ + Topic: topicName, + SubscriptionName: "sub-1", + Type: Exclusive, + }) + assert.Nil(t, err) + + // c2 is used to test if previous batch can be acked + // when cumulative ack the next batch message id + c2, err := client.Subscribe(ConsumerOptions{ + Topic: topicName, + SubscriptionName: "sub-2", + Type: Exclusive, + }) + assert.Nil(t, err) + + const N = 100 + + // send a batch + wg := sync.WaitGroup{} + for i := 0; i < N; i++ { + wg.Add(1) + producer.SendAsync(ctx, &ProducerMessage{ + Payload: []byte(fmt.Sprintf("msg-content-%d", i))}, + func(id MessageID, producerMessage *ProducerMessage, e error) { + assert.NoError(t, e) + wg.Done() + }) + } + wg.Wait() + + err = producer.Flush() + assert.NoError(t, err) + + // send another batch + wg = sync.WaitGroup{} + for i := N; i < 2*N; i++ { + wg.Add(1) + producer.SendAsync(ctx, &ProducerMessage{ + Payload: []byte(fmt.Sprintf("msg-content-%d", i))}, + func(id MessageID, producerMessage *ProducerMessage, e error) { + assert.NoError(t, e) + wg.Done() + }) + } + wg.Wait() + + for i := 0; i < 2*N; i++ { + msg, err := c1.Receive(ctx) + assert.Nil(t, err) + assert.Equal(t, fmt.Sprintf("msg-content-%d", i), string(msg.Payload())) + + if i == N-1 { + // cumulative ack the first half of messages + c1.AckCumulative(msg) + } else if i == N { + // the N+1 msg is in the second batch + // cumulative ack it to test if the first batch can be acked + c2.AckCumulative(msg) + } + } + + c1.Close() + c2.Close() + + // Subscribe again + c1, err = client.Subscribe(ConsumerOptions{ + Topic: topicName, + SubscriptionName: "sub-1", + Type: Exclusive, + }) + assert.Nil(t, err) + defer c1.Close() + + // Subscribe again + c2, err = client.Subscribe(ConsumerOptions{ + Topic: topicName, + SubscriptionName: "sub-2", + Type: Exclusive, + }) + assert.Nil(t, err) + defer c2.Close() + + // We should only receive the 2nd half of messages + for i := N; i < 2*N; i++ { + msg, err := c1.Receive(ctx) + assert.Nil(t, err) + assert.Equal(t, fmt.Sprintf("msg-content-%d", i), string(msg.Payload())) + + c1.Ack(msg) + } + + // We should only receive the 2nd half of messages + for i := N; i < 2*N; i++ { + msg, err := c2.Receive(ctx) + assert.Nil(t, err) + assert.Equal(t, fmt.Sprintf("msg-content-%d", i), string(msg.Payload())) + + c2.Ack(msg) + } +} + func TestConsumerNack(t *testing.T) { client, err := NewClient(ClientOptions{ URL: lookupURL, @@ -1573,6 +1760,43 @@ func TestAckWithResponse(t *testing.T) { } } +func TestCumulativeAckWithResponse(t *testing.T) { + now := time.Now().Unix() + topic01 := fmt.Sprintf("persistent://public/default/topic-%d-01", now) + ctx := context.Background() + + client, err := NewClient(ClientOptions{URL: lookupURL}) + assert.Nil(t, err) + defer client.Close() + + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic01, + SubscriptionName: "my-sub", + Type: Exclusive, + SubscriptionInitialPosition: SubscriptionPositionEarliest, + AckWithResponse: true, + }) + assert.Nil(t, err) + defer consumer.Close() + + producer01, err := client.CreateProducer(ProducerOptions{Topic: topic01}) + assert.Nil(t, err) + defer producer01.Close() + for i := 0; i < 10; i++ { + _, err = producer01.Send(ctx, &ProducerMessage{Payload: []byte(fmt.Sprintf("MSG_01_%d", i))}) + assert.Nil(t, err) + } + + var msg Message + for i := 0; i < 10; i++ { + msg, err = consumer.Receive(ctx) + assert.Nil(t, err) + } + + err = consumer.AckCumulative(msg) + assert.Nil(t, err) +} + func TestRLQMultiTopics(t *testing.T) { now := time.Now().Unix() topic01 := fmt.Sprintf("persistent://public/default/topic-%d-1", now) diff --git a/pulsar/impl_message.go b/pulsar/impl_message.go index 3a7feee9c3..d863da9bad 100644 --- a/pulsar/impl_message.go +++ b/pulsar/impl_message.go @@ -24,6 +24,7 @@ import ( "math/big" "strings" "sync" + "sync/atomic" "time" "google.golang.org/protobuf/proto" @@ -107,6 +108,25 @@ func (id trackingMessageID) ack() bool { return true } +func (id trackingMessageID) ackCumulative() bool { + if id.tracker != nil && id.batchIdx > -1 { + return id.tracker.ackCumulative(int(id.batchIdx)) + } + return true +} + +func (id trackingMessageID) prev() trackingMessageID { + return trackingMessageID{ + messageID: messageID{ + ledgerID: id.ledgerID, + entryID: id.entryID - 1, + partitionIdx: id.partitionIdx, + }, + tracker: id.tracker, + consumer: id.consumer, + } +} + func (id messageID) isEntryIDValid() bool { return id.entryID >= 0 } @@ -367,8 +387,9 @@ func newAckTracker(size int) *ackTracker { type ackTracker struct { sync.Mutex - size int - batchIDs *big.Int + size int + batchIDs *big.Int + prevBatchAcked uint32 } func (t *ackTracker) ack(batchID int) bool { @@ -381,6 +402,25 @@ func (t *ackTracker) ack(batchID int) bool { return len(t.batchIDs.Bits()) == 0 } +func (t *ackTracker) ackCumulative(batchID int) bool { + if batchID < 0 { + return true + } + mask := big.NewInt(-1) + t.Lock() + defer t.Unlock() + t.batchIDs.And(t.batchIDs, mask.Lsh(mask, uint(batchID+1))) + return len(t.batchIDs.Bits()) == 0 +} + +func (t *ackTracker) hasPrevBatchAcked() bool { + return atomic.LoadUint32(&t.prevBatchAcked) == 1 +} + +func (t *ackTracker) setPrevBatchAcked() { + atomic.StoreUint32(&t.prevBatchAcked, 1) +} + func (t *ackTracker) completed() bool { t.Lock() defer t.Unlock() diff --git a/pulsar/impl_message_test.go b/pulsar/impl_message_test.go index 3421ab4a43..89aab8a982 100644 --- a/pulsar/impl_message_test.go +++ b/pulsar/impl_message_test.go @@ -80,6 +80,23 @@ func TestAckTracker(t *testing.T) { } assert.Equal(t, true, tracker.completed()) + + tracker = newAckTracker(1000) + for i := 0; i < 1000; i++ { + if i < 999 { + assert.False(t, tracker.ackCumulative(i)) + assert.False(t, tracker.completed()) + } else { + assert.True(t, tracker.ackCumulative(i)) + assert.True(t, tracker.completed()) + } + } + + // test large number 1000 cumulative + tracker = newAckTracker(1000) + + assert.Equal(t, true, tracker.ackCumulative(999)) + assert.Equal(t, true, tracker.completed()) } func TestAckingMessageIDBatchOne(t *testing.T) { diff --git a/pulsar/internal/pulsartracing/consumer_interceptor_test.go b/pulsar/internal/pulsartracing/consumer_interceptor_test.go index 9e70d8bdf0..a06e4ee1d1 100644 --- a/pulsar/internal/pulsartracing/consumer_interceptor_test.go +++ b/pulsar/internal/pulsartracing/consumer_interceptor_test.go @@ -72,6 +72,14 @@ func (c *mockConsumer) AckID(msgID pulsar.MessageID) error { return nil } +func (c *mockConsumer) AckCumulative(msg pulsar.Message) error { + return nil +} + +func (c *mockConsumer) AckIDCumulative(msgID pulsar.MessageID) error { + return nil +} + func (c *mockConsumer) ReconsumeLater(msg pulsar.Message, delay time.Duration) {} func (c *mockConsumer) Nack(msg pulsar.Message) {} From 9fc96d429659648d00e3082eb060bb5d05c51fd7 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Tue, 3 Jan 2023 16:40:46 +0800 Subject: [PATCH 005/348] Handle nil value message correctly in table-view (#930) * Handle nil value message correct in table-view * Use require.Eventually --- pulsar/table_view_impl.go | 14 ++++++--- pulsar/table_view_test.go | 65 +++++++++++++++++++++++++++++++++++++++ 2 files changed, 75 insertions(+), 4 deletions(-) diff --git a/pulsar/table_view_impl.go b/pulsar/table_view_impl.go index 56b9fcbe04..85499f193b 100644 --- a/pulsar/table_view_impl.go +++ b/pulsar/table_view_impl.go @@ -245,11 +245,17 @@ func (tv *TableViewImpl) handleMessage(msg Message) { tv.dataMu.Lock() defer tv.dataMu.Unlock() - payload := reflect.Indirect(reflect.New(tv.options.SchemaValueType)).Interface() - if err := msg.GetSchemaValue(&payload); err != nil { - tv.logger.Errorf("msg.GetSchemaValue() failed with %w; msg is %v", msg, err) + var payload interface{} + if len(msg.Payload()) == 0 { + delete(tv.data, msg.Key()) + } else { + payload = reflect.Indirect(reflect.New(tv.options.SchemaValueType)).Interface() + if err := msg.GetSchemaValue(&payload); err != nil { + tv.logger.Errorf("msg.GetSchemaValue() failed with %w; msg is %v", msg, err) + } + tv.data[msg.Key()] = payload } - tv.data[msg.Key()] = payload + for _, listener := range tv.listeners { if err := listener(msg.Key(), payload); err != nil { tv.logger.Errorf("table view listener failed for %v: %w", msg, err) diff --git a/pulsar/table_view_test.go b/pulsar/table_view_test.go index 829187bd3c..d29b24d298 100644 --- a/pulsar/table_view_test.go +++ b/pulsar/table_view_test.go @@ -25,6 +25,7 @@ import ( "time" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestTableView(t *testing.T) { @@ -78,3 +79,67 @@ func TestTableView(t *testing.T) { assert.Equal(t, valuePrefix+k, *(v.(*string))) } } + +func TestPublishNilValue(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.NoError(t, err) + defer client.Close() + + topic := newTopicName() + schema := NewStringSchema(nil) + + // create producer + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + Schema: schema, + }) + assert.NoError(t, err) + defer producer.Close() + + // create table view + v := "" + tv, err := client.CreateTableView(TableViewOptions{ + Topic: topic, + Schema: schema, + SchemaValueType: reflect.TypeOf(&v), + }) + assert.NoError(t, err) + defer tv.Close() + + _, err = producer.Send(context.Background(), &ProducerMessage{ + Key: "key-1", + Value: "value-1", + }) + assert.NoError(t, err) + + require.Eventually(t, func() bool { + return tv.Size() == 1 + }, 5*time.Second, 100*time.Millisecond) + + assert.Equal(t, *(tv.Get("key-1").(*string)), "value-1") + + // send nil value + _, err = producer.Send(context.Background(), &ProducerMessage{ + Key: "key-1", + }) + assert.NoError(t, err) + + require.Eventually(t, func() bool { + return tv.Size() == 0 + }, 5*time.Second, 100*time.Millisecond) + + _, err = producer.Send(context.Background(), &ProducerMessage{ + Key: "key-2", + Value: "value-2", + }) + assert.NoError(t, err) + + require.Eventually(t, func() bool { + return tv.Size() == 1 + }, 5*time.Second, 100*time.Millisecond) + + assert.Equal(t, *(tv.Get("key-2").(*string)), "value-2") +} From 1f3747edc0f686f07e0bf2ab0ecc6709d9d5df30 Mon Sep 17 00:00:00 2001 From: Nitin Goyal Date: Tue, 3 Jan 2023 14:23:15 +0530 Subject: [PATCH 006/348] Allow CustomProperties when sending message for retry (#916) * Allow CustomProperties when sending message for retry * Lint issue fixed * TestRLQWithCustomProperties test case added Signed-off-by: Nitin Goyal --- pulsar/consumer.go | 3 + pulsar/consumer_impl.go | 10 ++ pulsar/consumer_multitopic.go | 7 +- pulsar/consumer_regex.go | 5 + pulsar/consumer_test.go | 114 ++++++++++++++++++ .../consumer_interceptor_test.go | 4 + 6 files changed, 142 insertions(+), 1 deletion(-) diff --git a/pulsar/consumer.go b/pulsar/consumer.go index 49a24d56d6..c9f5e89393 100644 --- a/pulsar/consumer.go +++ b/pulsar/consumer.go @@ -245,6 +245,9 @@ type Consumer interface { // ReconsumeLater mark a message for redelivery after custom delay ReconsumeLater(msg Message, delay time.Duration) + // ReconsumeLaterWithCustomProperties mark a message for redelivery after custom delay with custom properties + ReconsumeLaterWithCustomProperties(msg Message, customProperties map[string]string, delay time.Duration) + // Nack acknowledges the failure to process a single message. // // When a message is "negatively acked" it will be marked for redelivery after diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index 11a3cf70af..545b57c225 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -520,6 +520,12 @@ func (c *consumer) AckIDCumulative(msgID MessageID) error { // ReconsumeLater mark a message for redelivery after custom delay func (c *consumer) ReconsumeLater(msg Message, delay time.Duration) { + c.ReconsumeLaterWithCustomProperties(msg, map[string]string{}, delay) +} + +// ReconsumeLaterWithCustomProperties mark a message for redelivery after custom delay with custom properties +func (c *consumer) ReconsumeLaterWithCustomProperties(msg Message, customProperties map[string]string, + delay time.Duration) { if delay < 0 { delay = 0 } @@ -532,6 +538,10 @@ func (c *consumer) ReconsumeLater(msg Message, delay time.Duration) { props[k] = v } + for k, v := range customProperties { + props[k] = v + } + reconsumeTimes := 1 if s, ok := props[SysPropertyReconsumeTimes]; ok { reconsumeTimes, _ = strconv.Atoi(s) diff --git a/pulsar/consumer_multitopic.go b/pulsar/consumer_multitopic.go index e653926322..452915a87e 100644 --- a/pulsar/consumer_multitopic.go +++ b/pulsar/consumer_multitopic.go @@ -171,6 +171,11 @@ func (c *multiTopicConsumer) AckIDCumulative(msgID MessageID) error { } func (c *multiTopicConsumer) ReconsumeLater(msg Message, delay time.Duration) { + c.ReconsumeLaterWithCustomProperties(msg, map[string]string{}, delay) +} + +func (c *multiTopicConsumer) ReconsumeLaterWithCustomProperties(msg Message, customProperties map[string]string, + delay time.Duration) { names, err := validateTopicNames(msg.Topic()) if err != nil { c.log.Errorf("validate msg topic %q failed: %v", msg.Topic(), err) @@ -192,7 +197,7 @@ func (c *multiTopicConsumer) ReconsumeLater(msg Message, delay time.Duration) { return } } - consumer.ReconsumeLater(msg, delay) + consumer.ReconsumeLaterWithCustomProperties(msg, customProperties, delay) } func (c *multiTopicConsumer) Nack(msg Message) { diff --git a/pulsar/consumer_regex.go b/pulsar/consumer_regex.go index 36d2245f26..d890c67306 100644 --- a/pulsar/consumer_regex.go +++ b/pulsar/consumer_regex.go @@ -167,6 +167,11 @@ func (c *regexConsumer) ReconsumeLater(msg Message, delay time.Duration) { c.log.Warnf("regexp consumer not support ReconsumeLater yet.") } +func (c *regexConsumer) ReconsumeLaterWithCustomProperties(msg Message, customProperties map[string]string, + delay time.Duration) { + c.log.Warnf("regexp consumer not support ReconsumeLaterWithCustomProperties yet.") +} + // AckID the consumption of a single message, identified by its MessageID func (c *regexConsumer) AckID(msgID MessageID) error { mid, ok := toTrackingMessageID(msgID) diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 891fcd23c1..11e72a7240 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -1725,6 +1725,120 @@ func TestRLQ(t *testing.T) { assert.Nil(t, checkMsg) } +func TestRLQWithCustomProperties(t *testing.T) { + topic := newTopicName() + testURL := adminURL + "/" + "admin/v2/persistent/public/default/" + topic + "/partitions" + makeHTTPCall(t, http.MethodPut, testURL, "3") + + subName := fmt.Sprintf("sub01-%d", time.Now().Unix()) + maxRedeliveries := 2 + N := 100 + ctx := context.Background() + + client, err := NewClient(ClientOptions{URL: lookupURL}) + assert.Nil(t, err) + defer client.Close() + + // 1. Pre-produce N messages + producer, err := client.CreateProducer(ProducerOptions{Topic: topic}) + assert.Nil(t, err) + defer producer.Close() + + for i := 0; i < N; i++ { + _, err = producer.Send(ctx, &ProducerMessage{Payload: []byte(fmt.Sprintf("MESSAGE_%d", i))}) + assert.Nil(t, err) + } + + // 2. Create consumer on the Retry Topic to reconsume N messages (maxRedeliveries+1) times + rlqConsumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: subName, + Type: Shared, + SubscriptionInitialPosition: SubscriptionPositionEarliest, + DLQ: &DLQPolicy{ + MaxDeliveries: uint32(maxRedeliveries), + }, + RetryEnable: true, + NackRedeliveryDelay: 1 * time.Second, + }) + assert.Nil(t, err) + defer rlqConsumer.Close() + + rlqReceived := 0 + for rlqReceived < N*(maxRedeliveries+1) { + msg, err := rlqConsumer.Receive(ctx) + assert.Nil(t, err) + + if msg.RedeliveryCount() > 0 { + msgProps := msg.Properties() + + value, ok := msgProps["custom-key-1"] + assert.True(t, ok) + if ok { + assert.Equal(t, value, "custom-value-1") + } + + rlqConsumer.ReconsumeLater(msg, 1*time.Second) + } else { + customProps := map[string]string{ + "custom-key-1": "custom-val-1", + } + rlqConsumer.ReconsumeLaterWithCustomProperties(msg, customProps, 1*time.Second) + } + + rlqReceived++ + } + fmt.Println("retry consumed:", rlqReceived) // 300 + + // No more messages on the Retry Topic + rlqCtx, rlqCancel := context.WithTimeout(context.Background(), 500*time.Millisecond) + defer rlqCancel() + msg, err := rlqConsumer.Receive(rlqCtx) + assert.Error(t, err) + assert.Nil(t, msg) + + // 3. Create consumer on the DLQ topic to verify the routing + dlqConsumer, err := client.Subscribe(ConsumerOptions{ + Topic: "persistent://public/default/" + topic + "-" + subName + "-DLQ", + SubscriptionName: subName, + SubscriptionInitialPosition: SubscriptionPositionEarliest, + }) + assert.Nil(t, err) + defer dlqConsumer.Close() + + dlqReceived := 0 + for dlqReceived < N { + msg, err := dlqConsumer.Receive(ctx) + assert.Nil(t, err) + dlqConsumer.Ack(msg) + dlqReceived++ + } + fmt.Println("dlq received:", dlqReceived) // 100 + + // No more messages on the DLQ Topic + dlqCtx, dlqCancel := context.WithTimeout(context.Background(), 500*time.Millisecond) + defer dlqCancel() + msg, err = dlqConsumer.Receive(dlqCtx) + assert.Error(t, err) + assert.Nil(t, msg) + + // 4. No more messages for same subscription + checkConsumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: subName, + Type: Shared, + SubscriptionInitialPosition: SubscriptionPositionEarliest, + }) + assert.Nil(t, err) + defer checkConsumer.Close() + + checkCtx, checkCancel := context.WithTimeout(context.Background(), 500*time.Millisecond) + defer checkCancel() + checkMsg, err := checkConsumer.Receive(checkCtx) + assert.Error(t, err) + assert.Nil(t, checkMsg) +} + func TestAckWithResponse(t *testing.T) { now := time.Now().Unix() topic01 := fmt.Sprintf("persistent://public/default/topic-%d-01", now) diff --git a/pulsar/internal/pulsartracing/consumer_interceptor_test.go b/pulsar/internal/pulsartracing/consumer_interceptor_test.go index a06e4ee1d1..34e09d58a9 100644 --- a/pulsar/internal/pulsartracing/consumer_interceptor_test.go +++ b/pulsar/internal/pulsartracing/consumer_interceptor_test.go @@ -82,6 +82,10 @@ func (c *mockConsumer) AckIDCumulative(msgID pulsar.MessageID) error { func (c *mockConsumer) ReconsumeLater(msg pulsar.Message, delay time.Duration) {} +func (c *mockConsumer) ReconsumeLaterWithCustomProperties(msg pulsar.Message, customProperties map[string]string, + delay time.Duration) { +} + func (c *mockConsumer) Nack(msg pulsar.Message) {} func (c *mockConsumer) NackID(msgID pulsar.MessageID) {} From 44dc85c9323f5e64d1acf250fcafe8580617ce52 Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Tue, 3 Jan 2023 20:50:45 +0800 Subject: [PATCH 007/348] [Issue 918] [Refactor] Remove the clearMessageQueuesCh in partitionConsumer.dispatcher() (#921) Master Issue: #918 ### Motivation The two chanel `clearMessageQueuesCh` and `clearQueueCb` just need to keep one. For more details please check #918. This PR does not ony aim to clean up, but also fix the potential bug in `clearMessageQueuesCh`. For example, the `clearMessageQueuesCh` do the jod including clearing messageCh. But it may cause problem when `SeekByTime` invoked on partition topic. /~https://github.com/apache/pulsar-client-go/blob/1d3499a18d526b4b1aef0bdbbc54ac812b8ae0c0/pulsar/consumer_impl.go#L614-L626 /~https://github.com/apache/pulsar-client-go/blob/1d3499a18d526b4b1aef0bdbbc54ac812b8ae0c0/pulsar/consumer_partition.go#L1168-L1175 When consume the partition topic, all the `partitionConsumer` share the same `messageCh`. After `SeekByTime` on partitioned topic, the messageCh may be cleared more than one time which will cause the messages losing. Suppose there is such a situation, partitionConsumer-1 has cleared its messageCh and queueCh. When partitionConsumer-2 do the clear job, it can also exec this logic. /~https://github.com/apache/pulsar-client-go/blob/1d3499a18d526b4b1aef0bdbbc54ac812b8ae0c0/pulsar/consumer_partition.go#L1172-L1174 But `messageCh` is a share chan, partitionConsumer-1 may received new messages and put them to `messageCh` at this moment. There is such a possibility that partitionConsumer-2 **cleared the new messages** from `messageCh`. ### Modifications - Remove the `clearMessageQueuesCh` in `partitionConsumer.dispatcher()` - Modify the `clearQueueCb` in `partitionConsumer.dispatcher()` --- pulsar/consumer_impl.go | 17 ++++++- pulsar/consumer_partition.go | 88 ++++++++++++++++-------------------- pulsar/reader_impl.go | 6 ++- 3 files changed, 60 insertions(+), 51 deletions(-) diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index 545b57c225..e0120ad53e 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -640,7 +640,16 @@ func (c *consumer) Seek(msgID MessageID) error { return err } - return c.consumers[msgID.PartitionIdx()].Seek(msgID) + if err := c.consumers[msgID.PartitionIdx()].Seek(msgID); err != nil { + return err + } + + // clear messageCh + for len(c.messageCh) > 0 { + <-c.messageCh + } + + return nil } func (c *consumer) SeekByTime(time time.Time) error { @@ -654,6 +663,12 @@ func (c *consumer) SeekByTime(time time.Time) error { errs = pkgerrors.Wrap(newError(SeekFailed, err.Error()), msg) } } + + // clear messageCh + for len(c.messageCh) > 0 { + <-c.messageCh + } + return errs } diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 7f5b6df711..5ee09abce4 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -145,15 +145,14 @@ type partitionConsumer struct { // the size of the queue channel for buffering messages queueSize int32 queueCh chan []*message - startMessageID trackingMessageID + startMessageID atomicMessageID lastDequeuedMsg trackingMessageID - eventsCh chan interface{} - connectedCh chan struct{} - connectClosedCh chan connectionClosed - closeCh chan struct{} - clearQueueCh chan func(id trackingMessageID) - clearMessageQueuesCh chan chan struct{} + eventsCh chan interface{} + connectedCh chan struct{} + connectClosedCh chan connectionClosed + closeCh chan struct{} + clearQueueCh chan func(id trackingMessageID) nackTracker *negativeAcksTracker dlq *dlqRouter @@ -212,6 +211,24 @@ func (p *availablePermits) reset() { atomic.StoreInt32(&p.permits, 0) } +// atomicMessageID is a wrapper for trackingMessageID to make get and set atomic +type atomicMessageID struct { + msgID trackingMessageID + sync.RWMutex +} + +func (a *atomicMessageID) get() trackingMessageID { + a.RLock() + defer a.RUnlock() + return a.msgID +} + +func (a *atomicMessageID) set(msgID trackingMessageID) { + a.Lock() + defer a.Unlock() + a.msgID = msgID +} + type schemaInfoCache struct { lock sync.RWMutex cache map[string]Schema @@ -277,13 +294,12 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon eventsCh: make(chan interface{}, 10), queueSize: int32(options.receiverQueueSize), queueCh: make(chan []*message, options.receiverQueueSize), - startMessageID: options.startMessageID, + startMessageID: atomicMessageID{msgID: options.startMessageID}, connectedCh: make(chan struct{}), messageCh: messageCh, connectClosedCh: make(chan connectionClosed, 10), closeCh: make(chan struct{}), clearQueueCh: make(chan func(id trackingMessageID)), - clearMessageQueuesCh: make(chan chan struct{}), compressionProviders: sync.Map{}, dlq: dlq, metrics: metrics, @@ -324,14 +340,14 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon pc.log.Info("Created consumer") pc.setConsumerState(consumerReady) - if pc.options.startMessageIDInclusive && pc.startMessageID.equal(lastestMessageID.(messageID)) { + if pc.options.startMessageIDInclusive && pc.startMessageID.get().equal(lastestMessageID.(messageID)) { msgID, err := pc.requestGetLastMessageID() if err != nil { pc.nackTracker.Close() return nil, err } if msgID.entryID != noMessageEntry { - pc.startMessageID = msgID + pc.startMessageID.set(msgID) // use the WithoutClear version because the dispatcher is not started yet err = pc.requestSeekWithoutClear(msgID.messageID) @@ -671,7 +687,7 @@ func (pc *partitionConsumer) requestSeek(msgID messageID) error { if err := pc.requestSeekWithoutClear(msgID); err != nil { return err } - pc.clearMessageChannels() + pc.clearReceiverQueue() return nil } @@ -742,13 +758,7 @@ func (pc *partitionConsumer) internalSeekByTime(seek *seekByTimeRequest) { seek.err = err return } - pc.clearMessageChannels() -} - -func (pc *partitionConsumer) clearMessageChannels() { - doneCh := make(chan struct{}) - pc.clearMessageQueuesCh <- doneCh - <-doneCh + pc.clearReceiverQueue() } func (pc *partitionConsumer) internalAck(req *ackRequest) { @@ -1057,7 +1067,7 @@ func (pc *partitionConsumer) processMessageChunk(compressedPayload internal.Buff } func (pc *partitionConsumer) messageShouldBeDiscarded(msgID trackingMessageID) bool { - if pc.startMessageID.Undefined() { + if pc.startMessageID.get().Undefined() { return false } // if we start at latest message, we should never discard @@ -1066,11 +1076,11 @@ func (pc *partitionConsumer) messageShouldBeDiscarded(msgID trackingMessageID) b } if pc.options.startMessageIDInclusive { - return pc.startMessageID.greater(msgID.messageID) + return pc.startMessageID.get().greater(msgID.messageID) } // Non inclusive - return pc.startMessageID.greaterEqual(msgID.messageID) + return pc.startMessageID.get().greaterEqual(msgID.messageID) } // create EncryptionContext from message metadata @@ -1218,37 +1228,19 @@ func (pc *partitionConsumer) dispatcher() { go func() { pc.queueCh <- nil }() + for m := range pc.queueCh { // the queue has been drained if m == nil { break } else if nextMessageInQueue.Undefined() { - nextMessageInQueue = m[0].msgID.(trackingMessageID) + nextMessageInQueue, _ = toTrackingMessageID(m[0].msgID) } } - clearQueueCb(nextMessageInQueue) - - case doneCh := <-pc.clearMessageQueuesCh: - for len(pc.queueCh) > 0 { - <-pc.queueCh - } - for len(pc.messageCh) > 0 { - <-pc.messageCh - } messages = nil - // reset available permits - pc.availablePermits.reset() - initialPermits := uint32(pc.queueSize) - - pc.log.Debugf("dispatcher requesting initial permits=%d", initialPermits) - // send initial permits - if err := pc.internalFlow(initialPermits); err != nil { - pc.log.WithError(err).Error("unable to send initial permits to broker") - } - - close(doneCh) + clearQueueCb(nextMessageInQueue) } } } @@ -1490,10 +1482,10 @@ func (pc *partitionConsumer) grabConn() error { KeySharedMeta: keySharedMeta, } - pc.startMessageID = pc.clearReceiverQueue() + pc.startMessageID.set(pc.clearReceiverQueue()) if pc.options.subscriptionMode != durable { // For regular subscriptions the broker will determine the restarting point - cmdSubscribe.StartMessageId = convertToMessageIDData(pc.startMessageID) + cmdSubscribe.StartMessageId = convertToMessageIDData(pc.startMessageID.get()) } if len(pc.options.metadata) > 0 { @@ -1571,8 +1563,8 @@ func (pc *partitionConsumer) clearQueueAndGetNextMessage() trackingMessageID { func (pc *partitionConsumer) clearReceiverQueue() trackingMessageID { nextMessageInQueue := pc.clearQueueAndGetNextMessage() - if pc.startMessageID.Undefined() { - return pc.startMessageID + if pc.startMessageID.get().Undefined() { + return pc.startMessageID.get() } if !nextMessageInQueue.Undefined() { @@ -1583,7 +1575,7 @@ func (pc *partitionConsumer) clearReceiverQueue() trackingMessageID { return pc.lastDequeuedMsg } else { // No message was received or dequeued by this consumer. Next message would still be the startMessageId - return pc.startMessageID + return pc.startMessageID.get() } } diff --git a/pulsar/reader_impl.go b/pulsar/reader_impl.go index 3931f36c79..079754b0bf 100644 --- a/pulsar/reader_impl.go +++ b/pulsar/reader_impl.go @@ -185,11 +185,13 @@ func (r *reader) hasMoreMessages() bool { } if r.pc.options.startMessageIDInclusive { - return r.lastMessageInBroker.isEntryIDValid() && r.lastMessageInBroker.greaterEqual(r.pc.startMessageID.messageID) + return r.lastMessageInBroker.isEntryIDValid() && + r.lastMessageInBroker.greaterEqual(r.pc.startMessageID.get().messageID) } // Non-inclusive - return r.lastMessageInBroker.isEntryIDValid() && r.lastMessageInBroker.greater(r.pc.startMessageID.messageID) + return r.lastMessageInBroker.isEntryIDValid() && + r.lastMessageInBroker.greater(r.pc.startMessageID.get().messageID) } func (r *reader) Close() { From cf031b8951aec1356f2e064f1bf679d24c81ccd0 Mon Sep 17 00:00:00 2001 From: Garule Prabhudas Date: Tue, 10 Jan 2023 17:46:33 +0530 Subject: [PATCH 008/348] add messageId and topic as props of DLQ message (#907) Co-authored-by: Prabhudas Garule --- pulsar/consumer_test.go | 6 ++++++ pulsar/dlq_router.go | 12 +++++++++++- pulsar/message.go | 3 +++ pulsar/reader_test.go | 8 ++++++++ pulsar/retry_router.go | 1 + 5 files changed, 29 insertions(+), 1 deletion(-) diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 11e72a7240..2f1a0562b5 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -1505,6 +1505,12 @@ func DLQWithProducerOptions(t *testing.T, prodOpt *ProducerOptions) { expectMsg := fmt.Sprintf("hello-%d", expectedMsgIdx) assert.Equal(t, []byte(expectMsg), msg.Payload()) + + // check original messageId + assert.NotEmpty(t, msg.Properties()[PropertyOriginMessageID]) + + // check original topic + assert.NotEmpty(t, msg.Properties()[SysPropertyRealTopic]) } // No more messages on the DLQ diff --git a/pulsar/dlq_router.go b/pulsar/dlq_router.go index 000faaa0dd..5ecd8f8632 100644 --- a/pulsar/dlq_router.go +++ b/pulsar/dlq_router.go @@ -92,11 +92,21 @@ func (r *dlqRouter) run() { producer := r.getProducer(cm.Consumer.(*consumer).options.Schema) msg := cm.Message.(*message) msgID := msg.ID() + + // properties associated with original message + properties := msg.Properties() + + // include orinal message id in string format in properties + properties[PropertyOriginMessageID] = msgID.String() + + // include original topic name of the message in properties + properties[SysPropertyRealTopic] = msg.Topic() + producer.SendAsync(context.Background(), &ProducerMessage{ Payload: msg.Payload(), Key: msg.Key(), OrderingKey: msg.OrderingKey(), - Properties: msg.Properties(), + Properties: properties, EventTime: msg.EventTime(), ReplicationClusters: msg.replicationClusters, }, func(MessageID, *ProducerMessage, error) { diff --git a/pulsar/message.go b/pulsar/message.go index 7f2e07f8b6..76d017658c 100644 --- a/pulsar/message.go +++ b/pulsar/message.go @@ -154,6 +154,9 @@ type MessageID interface { // PartitionIdx returns the message partitionIdx PartitionIdx() int32 + + // String returns message id in string format + String() string } // DeserializeMessageID reconstruct a MessageID object from its serialized representation diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index 07e7ed3c9f..53bd459f8e 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -430,6 +430,14 @@ func (id *myMessageID) PartitionIdx() int32 { return id.PartitionIdx() } +func (id *myMessageID) String() string { + mid, err := DeserializeMessageID(id.data) + if err != nil { + return "" + } + return fmt.Sprintf("%d:%d:%d", mid.LedgerID(), mid.EntryID(), mid.PartitionIdx()) +} + func TestReaderOnSpecificMessageWithCustomMessageID(t *testing.T) { client, err := NewClient(ClientOptions{ URL: lookupURL, diff --git a/pulsar/retry_router.go b/pulsar/retry_router.go index 7b5f6b8900..75792adc14 100644 --- a/pulsar/retry_router.go +++ b/pulsar/retry_router.go @@ -35,6 +35,7 @@ const ( SysPropertyRetryTopic = "RETRY_TOPIC" SysPropertyReconsumeTimes = "RECONSUMETIMES" SysPropertyOriginMessageID = "ORIGIN_MESSAGE_IDY_TIME" + PropertyOriginMessageID = "ORIGIN_MESSAGE_ID" ) type RetryMessage struct { From 3cd02b1ce8ed15716aff9b04641109e11a704a80 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 11 Jan 2023 16:02:49 +0800 Subject: [PATCH 009/348] [feature] Support batch index ACK (#938) Fixes /~https://github.com/apache/pulsar-client-go/issues/894 ### Modifications Add an `EnableBatchIndexAcknowledgment` to specify whether batch index ACK is enabled. Since this feature requires the conversion between a bit set and its underlying long array, which is similar to Java's `BitSet`, this commit introduces github.com/bits-and-blooms/bitset dependency to replace the `big.Int` based implementation of the bit set. Add a `BatchSize()` method to `MessageId` to indicate the size of the `ack_set` field. When the batch index ACK happens, convert the `[]uint64` to the `[]int64` as the `ack_set` field in `CommandAck`. When receiving messages, convert the `ack_set` field in `CommandMessage` to filter the acknowledged single messages. Remove the duplicated code in `AckID` and `AckIDWithResponse`. ### Verifications `TestBatchIndexAck` is added to cover the case whether `AckWithResponse` is enabled and both individual and cumulative ACK. --- go.mod | 1 + go.sum | 2 + integration-tests/conf/standalone.conf | 2 + pulsar/consumer.go | 4 + pulsar/consumer_impl.go | 1 + pulsar/consumer_partition.go | 80 ++++++++------- pulsar/consumer_test.go | 135 +++++++++++++++++++++++++ pulsar/impl_message.go | 61 +++++++---- pulsar/impl_message_test.go | 16 +-- pulsar/message.go | 5 +- pulsar/producer_partition.go | 12 +++ pulsar/reader_test.go | 4 + 12 files changed, 259 insertions(+), 64 deletions(-) diff --git a/go.mod b/go.mod index 0435db9136..e143b042c9 100644 --- a/go.mod +++ b/go.mod @@ -6,6 +6,7 @@ require ( github.com/99designs/keyring v1.2.1 github.com/AthenZ/athenz v1.10.39 github.com/DataDog/zstd v1.5.0 + github.com/bits-and-blooms/bitset v1.4.0 github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b github.com/davecgh/go-spew v1.1.1 github.com/golang-jwt/jwt v3.2.1+incompatible diff --git a/go.sum b/go.sum index 9dbd99f101..6b07567812 100644 --- a/go.sum +++ b/go.sum @@ -65,6 +65,8 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= +github.com/bits-and-blooms/bitset v1.4.0 h1:+YZ8ePm+He2pU3dZlIZiOeAKfrBkXi1lSrXJ/Xzgbu8= +github.com/bits-and-blooms/bitset v1.4.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bketelsen/crypt v0.0.4/go.mod h1:aI6NrJ0pMGgvZKL1iVgXLnfIFJtfV+bKCoqOes/6LfM= github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b h1:AP/Y7sqYicnjGDfD5VcY4CIfh1hRXBUavxrvELjTiOE= github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b/go.mod h1:ac9efd0D1fsDb3EJvhqgXRbFx7bs2wqZ10HQPeU8U/Q= diff --git a/integration-tests/conf/standalone.conf b/integration-tests/conf/standalone.conf index 8cd28280ac..b9ff87bedb 100644 --- a/integration-tests/conf/standalone.conf +++ b/integration-tests/conf/standalone.conf @@ -292,3 +292,5 @@ globalZookeeperServers= # Deprecated. Use brokerDeleteInactiveTopicsFrequencySeconds brokerServicePurgeInactiveFrequencyInSeconds=60 + +acknowledgmentAtBatchIndexLevelEnabled=true diff --git a/pulsar/consumer.go b/pulsar/consumer.go index c9f5e89393..8bae57d950 100644 --- a/pulsar/consumer.go +++ b/pulsar/consumer.go @@ -211,6 +211,10 @@ type ConsumerOptions struct { // AutoAckIncompleteChunk sets whether consumer auto acknowledges incomplete chunked message when it should // be removed (e.g.the chunked message pending queue is full). (default: false) AutoAckIncompleteChunk bool + + // Enable or disable batch index acknowledgment. To enable this feature, ensure batch index acknowledgment + // is enabled on the broker side. (default: false) + EnableBatchIndexAcknowledgment bool } // Consumer is an interface that abstracts behavior of Pulsar's consumer diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index e0120ad53e..bf136c8e83 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -397,6 +397,7 @@ func (c *consumer) internalTopicSubscribeToPartitions() error { expireTimeOfIncompleteChunk: c.options.ExpireTimeOfIncompleteChunk, autoAckIncompleteChunk: c.options.AutoAckIncompleteChunk, consumerEventListener: c.options.EventListener, + enableBatchIndexAck: c.options.EnableBatchIndexAcknowledgment, } cons, err := newPartitionConsumer(c, c.client, opts, c.messageCh, c.dlq, c.metrics) ch <- ConsumerError{ diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 5ee09abce4..e723f8afa6 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -36,6 +36,7 @@ import ( cryptointernal "github.com/apache/pulsar-client-go/pulsar/internal/crypto" pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" "github.com/apache/pulsar-client-go/pulsar/log" + "github.com/bits-and-blooms/bitset" uAtomic "go.uber.org/atomic" ) @@ -114,6 +115,7 @@ type partitionConsumerOpts struct { autoAckIncompleteChunk bool // in failover mode, this callback will be called when consumer change consumerEventListener ConsumerEventListener + enableBatchIndexAck bool } type ConsumerEventListener interface { @@ -450,7 +452,7 @@ func (pc *partitionConsumer) requestGetLastMessageID() (trackingMessageID, error return convertToMessageID(id), nil } -func (pc *partitionConsumer) AckIDWithResponse(msgID MessageID) error { +func (pc *partitionConsumer) ackID(msgID MessageID, withResponse bool) error { if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing { pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer") return errors.New("consumer state is closed") @@ -474,47 +476,31 @@ func (pc *partitionConsumer) AckIDWithResponse(msgID MessageID) error { ackReq.msgID = trackingID // send ack request to eventsCh pc.eventsCh <- ackReq - // wait for the request to complete - <-ackReq.doneCh - - pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID) - } - - return ackReq.err -} - -func (pc *partitionConsumer) AckID(msgID MessageID) error { - if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing { - pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer") - return errors.New("consumer state is closed") - } - - if cmid, ok := toChunkedMessageID(msgID); ok { - return pc.unAckChunksTracker.ack(cmid) - } - trackingID, ok := toTrackingMessageID(msgID) - if !ok { - return errors.New("failed to convert trackingMessageID") - } + if withResponse { + <-ackReq.doneCh + } - ackReq := new(ackRequest) - ackReq.doneCh = make(chan struct{}) - ackReq.ackType = individualAck - if !trackingID.Undefined() && trackingID.ack() { - pc.metrics.AcksCounter.Inc() - pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9) + pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID) + } else if pc.options.enableBatchIndexAck { ackReq.msgID = trackingID - // send ack request to eventsCh pc.eventsCh <- ackReq - // No need to wait for ackReq.doneCh to finish - - pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID) } + if withResponse { + return ackReq.err + } return nil } +func (pc *partitionConsumer) AckIDWithResponse(msgID MessageID) error { + return pc.ackID(msgID, true) +} + +func (pc *partitionConsumer) AckID(msgID MessageID) error { + return pc.ackID(msgID, false) +} + func (pc *partitionConsumer) AckIDCumulative(msgID MessageID) error { return pc.internalAckIDCumulative(msgID, false) } @@ -541,7 +527,7 @@ func (pc *partitionConsumer) internalAckIDCumulative(msgID MessageID, withRespon ackReq := new(ackRequest) ackReq.doneCh = make(chan struct{}) ackReq.ackType = cumulativeAck - if trackingID.ackCumulative() { + if trackingID.ackCumulative() || pc.options.enableBatchIndexAck { ackReq.msgID = trackingID } else if !trackingID.tracker.hasPrevBatchAcked() { // get previous batch message id @@ -774,6 +760,12 @@ func (pc *partitionConsumer) internalAck(req *ackRequest) { LedgerId: proto.Uint64(uint64(msgID.ledgerID)), EntryId: proto.Uint64(uint64(msgID.entryID)), } + if pc.options.enableBatchIndexAck && msgID.tracker != nil { + ackSet := msgID.tracker.toAckSet() + if ackSet != nil { + messageIDs[0].AckSet = ackSet + } + } reqID := pc.client.rpcClient.NewRequestID() cmdAck := &pb.CommandAck{ @@ -832,7 +824,7 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header switch crypToFailureAction { case crypto.ConsumerCryptoFailureActionFail: pc.log.Errorf("consuming message failed due to decryption err :%v", err) - pc.NackID(newTrackingMessageID(int64(pbMsgID.GetLedgerId()), int64(pbMsgID.GetEntryId()), 0, 0, nil)) + pc.NackID(newTrackingMessageID(int64(pbMsgID.GetLedgerId()), int64(pbMsgID.GetEntryId()), 0, 0, 0, nil)) return err case crypto.ConsumerCryptoFailureActionDiscard: pc.discardCorruptedMessage(pbMsgID, pb.CommandAck_DecryptionError) @@ -852,6 +844,7 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header int64(pbMsgID.GetEntryId()), pbMsgID.GetBatchIndex(), pc.partitionIdx, + pbMsgID.GetBatchSize(), ), payLoad: headersAndPayload.ReadableSlice(), schema: pc.options.schema, @@ -899,7 +892,17 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header var ackTracker *ackTracker // are there multiple messages in this batch? if numMsgs > 1 { - ackTracker = newAckTracker(numMsgs) + ackTracker = newAckTracker(uint(numMsgs)) + } + + var ackSet *bitset.BitSet + if response.GetAckSet() != nil { + ackSetFromResponse := response.GetAckSet() + buf := make([]uint64, len(ackSetFromResponse)) + for i := 0; i < len(buf); i++ { + buf[i] = uint64(ackSetFromResponse[i]) + } + ackSet = bitset.From(buf) } pc.metrics.MessagesReceived.Add(float64(numMsgs)) @@ -911,6 +914,10 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header pc.discardCorruptedMessage(pbMsgID, pb.CommandAck_BatchDeSerializeError) return err } + if ackSet != nil && !ackSet.Test(uint(i)) { + pc.log.Debugf("Ignoring message from %vth message, which has been acknowledged", i) + continue + } pc.metrics.BytesReceived.Add(float64(len(payload))) pc.metrics.PrefetchedBytes.Add(float64(len(payload))) @@ -920,6 +927,7 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header int64(pbMsgID.GetEntryId()), int32(i), pc.partitionIdx, + int32(numMsgs), ackTracker) // set the consumer so we know how to ack the message id trackingMsgID.consumer = pc diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 2f1a0562b5..366be5de14 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -3851,3 +3851,138 @@ func TestAckWithMessageID(t *testing.T) { err = consumer.AckID(newID) assert.Nil(t, err) } + +func TestBatchIndexAck(t *testing.T) { + tests := []struct { + AckWithResponse bool + Cumulative bool + }{ + { + AckWithResponse: true, + Cumulative: true, + }, + { + AckWithResponse: true, + Cumulative: false, + }, + { + AckWithResponse: false, + Cumulative: true, + }, + { + AckWithResponse: false, + Cumulative: false, + }, + } + for _, params := range tests { + t.Run(fmt.Sprintf("TestBatchIndexAck_WithResponse_%v_Cumulative_%v", + params.AckWithResponse, params.Cumulative), + func(t *testing.T) { + runBatchIndexAckTest(t, params.AckWithResponse, params.Cumulative) + }) + } +} + +func runBatchIndexAckTest(t *testing.T, ackWithResponse bool, cumulative bool) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + + topic := newTopicName() + createConsumer := func() Consumer { + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + AckWithResponse: ackWithResponse, + EnableBatchIndexAcknowledgment: true, + }) + assert.Nil(t, err) + return consumer + } + + consumer := createConsumer() + + duration, err := time.ParseDuration("1h") + assert.Nil(t, err) + + const BatchingMaxSize int = 2 * 5 + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + DisableBatching: false, + BatchingMaxMessages: uint(BatchingMaxSize), + BatchingMaxSize: uint(1024 * 1024 * 10), + BatchingMaxPublishDelay: duration, + }) + assert.Nil(t, err) + for i := 0; i < BatchingMaxSize; i++ { + producer.SendAsync(context.Background(), &ProducerMessage{ + Payload: []byte(fmt.Sprintf("msg-%d", i)), + }, func(id MessageID, producerMessage *ProducerMessage, err error) { + assert.Nil(t, err) + log.Printf("Sent to %v:%d:%d", id, id.BatchIdx(), id.BatchSize()) + }) + } + assert.Nil(t, producer.Flush()) + + msgIds := make([]MessageID, BatchingMaxSize) + for i := 0; i < BatchingMaxSize; i++ { + message, err := consumer.Receive(context.Background()) + assert.Nil(t, err) + msgIds[i] = message.ID() + log.Printf("Received %v from %v:%d:%d", string(message.Payload()), message.ID(), + message.ID().BatchIdx(), message.ID().BatchSize()) + } + + // Acknowledge half of the messages + if cumulative { + msgID := msgIds[BatchingMaxSize/2-1] + consumer.AckIDCumulative(msgID) + log.Printf("Acknowledge %v:%d cumulatively\n", msgID, msgID.BatchIdx()) + } else { + for i := 0; i < BatchingMaxSize; i++ { + msgID := msgIds[i] + if i%2 == 0 { + consumer.AckID(msgID) + log.Printf("Acknowledge %v:%d\n", msgID, msgID.BatchIdx()) + } + } + } + consumer.Close() + consumer = createConsumer() + + for i := 0; i < BatchingMaxSize/2; i++ { + message, err := consumer.Receive(context.Background()) + assert.Nil(t, err) + log.Printf("Received %v from %v:%d:%d", string(message.Payload()), message.ID(), + message.ID().BatchIdx(), message.ID().BatchSize()) + index := i*2 + 1 + if cumulative { + index = i + BatchingMaxSize/2 + } + assert.Equal(t, []byte(fmt.Sprintf("msg-%d", index)), message.Payload()) + assert.Equal(t, msgIds[index].BatchIdx(), message.ID().BatchIdx()) + // We should not acknowledge message.ID() here because message.ID() shares a different + // tracker with msgIds + if !cumulative { + msgID := msgIds[index] + consumer.AckID(msgID) + log.Printf("Acknowledge %v:%d\n", msgID, msgID.BatchIdx()) + } + } + if cumulative { + msgID := msgIds[BatchingMaxSize-1] + consumer.AckIDCumulative(msgID) + log.Printf("Acknowledge %v:%d cumulatively\n", msgID, msgID.BatchIdx()) + } + consumer.Close() + consumer = createConsumer() + _, err = producer.Send(context.Background(), &ProducerMessage{Payload: []byte("end-marker")}) + assert.Nil(t, err) + msg, err := consumer.Receive(context.Background()) + assert.Nil(t, err) + assert.Equal(t, "end-marker", string(msg.Payload())) + + client.Close() +} diff --git a/pulsar/impl_message.go b/pulsar/impl_message.go index d863da9bad..39db8e1cb9 100644 --- a/pulsar/impl_message.go +++ b/pulsar/impl_message.go @@ -21,8 +21,6 @@ import ( "errors" "fmt" "math" - "math/big" - "strings" "sync" "sync/atomic" "time" @@ -30,6 +28,7 @@ import ( "google.golang.org/protobuf/proto" pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" + "github.com/bits-and-blooms/bitset" ) type messageID struct { @@ -37,6 +36,7 @@ type messageID struct { entryID int64 batchIdx int32 partitionIdx int32 + batchSize int32 } var latestMessageID = messageID{ @@ -44,6 +44,7 @@ var latestMessageID = messageID{ entryID: math.MaxInt64, batchIdx: -1, partitionIdx: -1, + batchSize: 0, } var earliestMessageID = messageID{ @@ -51,6 +52,7 @@ var earliestMessageID = messageID{ entryID: -1, batchIdx: -1, partitionIdx: -1, + batchSize: 0, } type trackingMessageID struct { @@ -159,6 +161,7 @@ func (id messageID) Serialize() []byte { EntryId: proto.Uint64(uint64(id.entryID)), BatchIndex: proto.Int32(id.batchIdx), Partition: proto.Int32(id.partitionIdx), + BatchSize: proto.Int32(id.batchSize), } data, _ := proto.Marshal(msgID) return data @@ -180,6 +183,10 @@ func (id messageID) PartitionIdx() int32 { return id.partitionIdx } +func (id messageID) BatchSize() int32 { + return id.batchSize +} + func (id messageID) String() string { return fmt.Sprintf("%d:%d:%d", id.ledgerID, id.entryID, id.partitionIdx) } @@ -195,20 +202,22 @@ func deserializeMessageID(data []byte) (MessageID, error) { int64(msgID.GetEntryId()), msgID.GetBatchIndex(), msgID.GetPartition(), + msgID.GetBatchSize(), ) return id, nil } -func newMessageID(ledgerID int64, entryID int64, batchIdx int32, partitionIdx int32) MessageID { +func newMessageID(ledgerID int64, entryID int64, batchIdx int32, partitionIdx int32, batchSize int32) MessageID { return messageID{ ledgerID: ledgerID, entryID: entryID, batchIdx: batchIdx, partitionIdx: partitionIdx, + batchSize: batchSize, } } -func newTrackingMessageID(ledgerID int64, entryID int64, batchIdx int32, partitionIdx int32, +func newTrackingMessageID(ledgerID int64, entryID int64, batchIdx int32, partitionIdx int32, batchSize int32, tracker *ackTracker) trackingMessageID { return trackingMessageID{ messageID: messageID{ @@ -216,6 +225,7 @@ func newTrackingMessageID(ledgerID int64, entryID int64, batchIdx int32, partiti entryID: entryID, batchIdx: batchIdx, partitionIdx: partitionIdx, + batchSize: batchSize, }, tracker: tracker, receivedTime: time.Now(), @@ -370,14 +380,10 @@ func (msg *message) BrokerPublishTime() *time.Time { return msg.brokerPublishTime } -func newAckTracker(size int) *ackTracker { - var batchIDs *big.Int - if size <= 64 { - shift := uint32(64 - size) - setBits := ^uint64(0) >> shift - batchIDs = new(big.Int).SetUint64(setBits) - } else { - batchIDs, _ = new(big.Int).SetString(strings.Repeat("1", size), 2) +func newAckTracker(size uint) *ackTracker { + batchIDs := bitset.New(size) + for i := uint(0); i < size; i++ { + batchIDs.Set(i) } return &ackTracker{ size: size, @@ -387,8 +393,8 @@ func newAckTracker(size int) *ackTracker { type ackTracker struct { sync.Mutex - size int - batchIDs *big.Int + size uint + batchIDs *bitset.BitSet prevBatchAcked uint32 } @@ -398,19 +404,20 @@ func (t *ackTracker) ack(batchID int) bool { } t.Lock() defer t.Unlock() - t.batchIDs = t.batchIDs.SetBit(t.batchIDs, batchID, 0) - return len(t.batchIDs.Bits()) == 0 + t.batchIDs.Clear(uint(batchID)) + return t.batchIDs.None() } func (t *ackTracker) ackCumulative(batchID int) bool { if batchID < 0 { return true } - mask := big.NewInt(-1) t.Lock() defer t.Unlock() - t.batchIDs.And(t.batchIDs, mask.Lsh(mask, uint(batchID+1))) - return len(t.batchIDs.Bits()) == 0 + for i := 0; i <= batchID; i++ { + t.batchIDs.Clear(uint(i)) + } + return t.batchIDs.None() } func (t *ackTracker) hasPrevBatchAcked() bool { @@ -424,7 +431,21 @@ func (t *ackTracker) setPrevBatchAcked() { func (t *ackTracker) completed() bool { t.Lock() defer t.Unlock() - return len(t.batchIDs.Bits()) == 0 + return t.batchIDs.None() +} + +func (t *ackTracker) toAckSet() []int64 { + t.Lock() + defer t.Unlock() + if t.batchIDs.None() { + return nil + } + bytes := t.batchIDs.Bytes() + ackSet := make([]int64, len(bytes)) + for i := 0; i < len(bytes); i++ { + ackSet[i] = int64(bytes[i]) + } + return ackSet } type chunkMessageID struct { diff --git a/pulsar/impl_message_test.go b/pulsar/impl_message_test.go index 89aab8a982..413a39fc37 100644 --- a/pulsar/impl_message_test.go +++ b/pulsar/impl_message_test.go @@ -24,7 +24,7 @@ import ( ) func TestMessageId(t *testing.T) { - id := newMessageID(1, 2, 3, 4) + id := newMessageID(1, 2, 3, 4, 5) bytes := id.Serialize() id2, err := DeserializeMessageID(bytes) @@ -35,6 +35,7 @@ func TestMessageId(t *testing.T) { assert.Equal(t, int64(2), id2.(messageID).entryID) assert.Equal(t, int32(3), id2.(messageID).batchIdx) assert.Equal(t, int32(4), id2.(messageID).partitionIdx) + assert.Equal(t, int32(5), id2.(messageID).batchSize) id, err = DeserializeMessageID(nil) assert.Error(t, err) @@ -47,11 +48,12 @@ func TestMessageId(t *testing.T) { func TestMessageIdGetFuncs(t *testing.T) { // test LedgerId,EntryId,BatchIdx,PartitionIdx - id := newMessageID(1, 2, 3, 4) + id := newMessageID(1, 2, 3, 4, 5) 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()) + assert.Equal(t, int32(5), id.BatchSize()) } func TestAckTracker(t *testing.T) { @@ -101,7 +103,7 @@ func TestAckTracker(t *testing.T) { func TestAckingMessageIDBatchOne(t *testing.T) { tracker := newAckTracker(1) - msgID := newTrackingMessageID(1, 1, 0, 0, tracker) + msgID := newTrackingMessageID(1, 1, 0, 0, 0, tracker) assert.Equal(t, true, msgID.ack()) assert.Equal(t, true, tracker.completed()) } @@ -109,8 +111,8 @@ func TestAckingMessageIDBatchOne(t *testing.T) { func TestAckingMessageIDBatchTwo(t *testing.T) { tracker := newAckTracker(2) ids := []trackingMessageID{ - newTrackingMessageID(1, 1, 0, 0, tracker), - newTrackingMessageID(1, 1, 1, 0, tracker), + newTrackingMessageID(1, 1, 0, 0, 0, tracker), + newTrackingMessageID(1, 1, 1, 0, 0, tracker), } assert.Equal(t, false, ids[0].ack()) @@ -120,8 +122,8 @@ func TestAckingMessageIDBatchTwo(t *testing.T) { // try reverse order tracker = newAckTracker(2) ids = []trackingMessageID{ - newTrackingMessageID(1, 1, 0, 0, tracker), - newTrackingMessageID(1, 1, 1, 0, tracker), + newTrackingMessageID(1, 1, 0, 0, 0, tracker), + newTrackingMessageID(1, 1, 1, 0, 0, tracker), } assert.Equal(t, false, ids[1].ack()) assert.Equal(t, true, ids[0].ack()) diff --git a/pulsar/message.go b/pulsar/message.go index 76d017658c..d37692b1a4 100644 --- a/pulsar/message.go +++ b/pulsar/message.go @@ -155,6 +155,9 @@ type MessageID interface { // PartitionIdx returns the message partitionIdx PartitionIdx() int32 + // BatchSize returns 0 or the batch size, which must be greater than BatchIdx() + BatchSize() int32 + // String returns message id in string format String() string } @@ -166,7 +169,7 @@ func DeserializeMessageID(data []byte) (MessageID, error) { // NewMessageID Custom Create MessageID func NewMessageID(ledgerID int64, entryID int64, batchIdx int32, partitionIdx int32) MessageID { - return newMessageID(ledgerID, entryID, batchIdx, partitionIdx) + return newMessageID(ledgerID, entryID, batchIdx, partitionIdx, 0) } // EarliestMessageID returns a messageID that points to the earliest message available in a topic diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 6308b55fc6..b0467f5409 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1118,6 +1118,15 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) pi.Lock() defer pi.Unlock() p.metrics.PublishRPCLatency.Observe(float64(now-pi.sentAt.UnixNano()) / 1.0e9) + batchSize := int32(0) + for _, i := range pi.sendRequests { + sr := i.(*sendRequest) + if sr.msg != nil { + batchSize = batchSize + 1 + } else { // Flush request + break + } + } for idx, i := range pi.sendRequests { sr := i.(*sendRequest) if sr.msg != nil { @@ -1138,6 +1147,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) int64(response.MessageId.GetEntryId()), int32(idx), p.partitionIdx, + batchSize, ) if sr.totalChunks > 1 { @@ -1148,6 +1158,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) int64(response.MessageId.GetEntryId()), -1, p.partitionIdx, + 0, }) } else if sr.chunkID == sr.totalChunks-1 { sr.chunkRecorder.setLastChunkID( @@ -1156,6 +1167,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) int64(response.MessageId.GetEntryId()), -1, p.partitionIdx, + 0, }) // use chunkMsgID to set msgID msgID = sr.chunkRecorder.chunkedMsgID diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index 53bd459f8e..35431874bc 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -426,6 +426,10 @@ func (id *myMessageID) BatchIdx() int32 { return id.BatchIdx() } +func (id *myMessageID) BatchSize() int32 { + return id.BatchSize() +} + func (id *myMessageID) PartitionIdx() int32 { return id.PartitionIdx() } From d9b18d0690c15d5da94c9f324d9c2618e3d60bfd Mon Sep 17 00:00:00 2001 From: Eugene R Date: Tue, 17 Jan 2023 09:27:40 +0200 Subject: [PATCH 010/348] Update the Cobra library to significantly reduce the dependency tree (#943) --- go.mod | 4 +- go.sum | 183 ++------------------------------------------------------- 2 files changed, 8 insertions(+), 179 deletions(-) diff --git a/go.mod b/go.mod index e143b042c9..fe6f41556b 100644 --- a/go.mod +++ b/go.mod @@ -21,7 +21,7 @@ require ( github.com/prometheus/client_golang v1.11.1 github.com/sirupsen/logrus v1.6.0 github.com/spaolacci/murmur3 v1.1.0 - github.com/spf13/cobra v1.2.1 + github.com/spf13/cobra v1.6.1 github.com/stretchr/testify v1.8.0 go.uber.org/atomic v1.7.0 golang.org/x/mod v0.5.1 @@ -41,7 +41,7 @@ require ( github.com/golang/protobuf v1.5.2 // indirect github.com/golang/snappy v0.0.1 // indirect github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect - github.com/inconshreveable/mousetrap v1.0.0 // indirect + github.com/inconshreveable/mousetrap v1.0.1 // indirect github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect github.com/mtibben/percent v0.2.1 // indirect diff --git a/go.sum b/go.sum index 6b07567812..4e96e3988e 100644 --- a/go.sum +++ b/go.sum @@ -13,11 +13,6 @@ cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKV cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= -cloud.google.com/go v0.72.0/go.mod h1:M+5Vjvlc2wnp6tjzE102Dw08nGShTscUx2nZMufOKPI= -cloud.google.com/go v0.74.0/go.mod h1:VV1xSbzvo+9QJOxLDaJfTjx5e+MePCpCWwvftOeQmWk= -cloud.google.com/go v0.78.0/go.mod h1:QjdrLG0uq+YwhjoVOLsS1t7TW8fs36kLs4XO5R5ECHg= -cloud.google.com/go v0.79.0/go.mod h1:3bzgcEeQlzbuEAYu4mrWhKqWjmpprinYgKJLgKHnbb8= -cloud.google.com/go v0.81.0/go.mod h1:mk/AM35KwGk/Nm2YSeZbxXdrNK3KZOYHmLkOqC2V6E0= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= @@ -26,7 +21,6 @@ cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4g cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= -cloud.google.com/go/firestore v1.1.0/go.mod h1:ulACoGHTpvq5r8rxGJ4ddJZBZqakUQqClKRT5SZwBmk= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -52,22 +46,16 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= -github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/ardielle/ardielle-go v1.5.2 h1:TilHTpHIQJ27R1Tl/iITBzMwiUGSlVfiVhwDNGM3Zj4= github.com/ardielle/ardielle-go v1.5.2/go.mod h1:I4hy1n795cUhaVt/ojz83SNVCYIGsAFAONtv2Dr7HUI= github.com/ardielle/ardielle-tools v1.5.4/go.mod h1:oZN+JRMnqGiIhrzkRN9l26Cej9dEx4jeNG6A+AdkShk= -github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= -github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= -github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/aws/aws-sdk-go v1.32.6/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= github.com/bits-and-blooms/bitset v1.4.0 h1:+YZ8ePm+He2pU3dZlIZiOeAKfrBkXi1lSrXJ/Xzgbu8= github.com/bits-and-blooms/bitset v1.4.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= -github.com/bketelsen/crypt v0.0.4/go.mod h1:aI6NrJ0pMGgvZKL1iVgXLnfIFJtfV+bKCoqOes/6LfM= github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b h1:AP/Y7sqYicnjGDfD5VcY4CIfh1hRXBUavxrvELjTiOE= github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b/go.mod h1:ac9efd0D1fsDb3EJvhqgXRbFx7bs2wqZ10HQPeU8U/Q= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= @@ -78,11 +66,7 @@ github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5P github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= -github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= -github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= +github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/danieljoos/wincred v1.1.2 h1:QLdCxFs1/Yl4zduvBdcHB8goaYk9RARS2SgLLRuAyr0= github.com/danieljoos/wincred v1.1.2/go.mod h1:GijpziifJoIBfYh+S7BbkdUTU4LfM+QnGqR5Vl2tAx0= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -95,15 +79,10 @@ github.com/dvsekhvalnov/jose2go v1.5.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/go-control-plane v0.9.7/go.mod h1:cwu0lG7PUMfa9snN8LXBig5ynNVH9qI8YYLbd1fK2po= -github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= -github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= -github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= @@ -118,9 +97,7 @@ github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/me github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 h1:ZpnhV/YsD2/4cESfV5+Hoeu/iUR3ruzNvZ+yQfO03a0= github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2/go.mod h1:bBOAhwG1umN6/6ZUMtDFBMQR8jRg9O75tm9K00oMsK4= -github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang-jwt/jwt v3.2.1+incompatible h1:73Z+4BJcrTC+KczS6WvTPvRGOp1WmfEP4Q1lOd9Z/+c= github.com/golang-jwt/jwt v3.2.1+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -134,7 +111,6 @@ github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= -github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -150,7 +126,6 @@ github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QD github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.1/go.mod h1:DopwsBzvsk0Fs44TXzsVbJyPhcCPeIwnvohx4u74HPM= github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= @@ -164,15 +139,12 @@ github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= -github.com/google/martian/v3 v3.1.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= @@ -180,45 +152,20 @@ github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hf github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20201023163331-3e6fc7fc9c4c/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= -github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= -github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c h1:6rhixN/i8ZofjG1Y75iExal34USq5p+wiN1tpie8IrU= github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c/go.mod h1:NMPJylDgVpX0MLRlPy15sqSwOFv/U1GZ2m21JhFfek0= -github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q= -github.com/hashicorp/consul/sdk v0.1.1/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= -github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= -github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= -github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= -github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= -github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= -github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= -github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= -github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= -github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= -github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= -github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= -github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= -github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= -github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= -github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= -github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/inconshreveable/mousetrap v1.0.1 h1:U3uMjPSQEBMNp1lFxmllqCPM6P5u/Xq7Pgzkat/bFNc= +github.com/inconshreveable/mousetrap v1.0.1/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= github.com/jawher/mow.cli v1.0.4/go.mod h1:5hQj2V8g+qYmLUVWqu4Wuja1pI57M83EChYLVZ0sMKk= github.com/jawher/mow.cli v1.2.0/go.mod h1:y+pcA3jBAdo/GIZx/0rFjw/K2bVEODP9rfZOfaiq8Ko= github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= @@ -228,17 +175,14 @@ github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/ github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= -github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= -github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.14.4 h1:eijASRJcobkVtSt81Olfh7JX43osYLwy5krOJo6YEu4= github.com/klauspost/compress v1.14.4/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= @@ -247,20 +191,8 @@ github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/linkedin/goavro/v2 v2.9.8 h1:jN50elxBsGBDGVDEKqUlDuU1cFwJ11K/yrJCBMe/7Wg= github.com/linkedin/goavro/v2 v2.9.8/go.mod h1:UgQUb2N/pmueQYH9bfqFioWxzYCZXSfF8Jw03O5sjqA= -github.com/magiconair/properties v1.8.5/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= -github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= -github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= -github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= -github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= -github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= -github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= -github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= -github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= -github.com/mitchellh/mapstructure v1.4.1/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= @@ -285,18 +217,14 @@ github.com/onsi/gomega v1.19.0 h1:4ieX6qQjPP/BfC3mpsAtIGGlxTWPeA3Inl/7DtXw1tw= github.com/onsi/gomega v1.19.0/go.mod h1:LY+I3pBVzYsTBU1AnDwOSxaYi9WoWiqgwooUqq9yPro= 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/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= -github.com/pelletier/go-toml v1.9.3/go.mod h1:u1nR/EPcESfeI/szUZKdtJ0xRNbUoANCkoOuaOx1Y+c= 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= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= @@ -316,28 +244,18 @@ github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsT github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= -github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= -github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= -github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= -github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= -github.com/spf13/cast v1.3.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v1.2.1 h1:+KmjbUw1hriSNMF55oPrkZcb27aECyrj8V2ytv7kWDw= -github.com/spf13/cobra v1.2.1/go.mod h1:ExllRjgxM/piMAM+3tAZvg8fsklGAf3tPfi+i8t68Nk= -github.com/spf13/jwalterweatherman v1.1.0/go.mod h1:aNWZUN0dPAAO/Ljvb5BEdw96iTZ0EXowPYD95IqWIGo= +github.com/spf13/cobra v1.6.1 h1:o94oiPyS4KD1mPy2fmcYYHHfCxLqYjJOhGsCHFZtEzA= +github.com/spf13/cobra v1.6.1/go.mod h1:IOw/AERYS7UzyrGinqmz6HLUo219MORXGxhbaJUqzrY= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= -github.com/spf13/viper v1.8.1/go.mod h1:o0Pch8wJ9BVSWGQMbra6iw0oQ5oktSIBaujf1rJH9Ns= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= @@ -347,38 +265,26 @@ 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/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= -github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PKk= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.etcd.io/etcd/api/v3 v3.5.0/go.mod h1:cbVKeC6lCfl7j/8jBhAK6aIYO9XOjdptoxU/nLQcPvs= -go.etcd.io/etcd/client/pkg/v3 v3.5.0/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= -go.etcd.io/etcd/client/v2 v2.305.0/go.mod h1:h9puh54ZTgAKtEbut2oe9P4L/oqKCVB6xsXlzd7alYQ= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= -go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/zap v1.17.0/go.mod h1:MXVU+bhUf/A7Xi2HNOnopQOrmycQ5Ih87HtOu4q5SSo= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -403,8 +309,6 @@ golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20201208152925-83fdc39ff7b5/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= @@ -413,17 +317,12 @@ golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzB golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.5.1 h1:OJxoQ/rynoF0dcCdI7cLPktw/hR2cueqYfjm43oqK38= golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -451,13 +350,6 @@ golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= -golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220225172249-27dd8689420f h1:oA4XRj0qtSt8Yo1Zms0CUlsT3KG69V2UGQWPBxujDmc= golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= @@ -466,12 +358,6 @@ golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4Iltr golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210313182246-cd4f82c27b84/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 h1:0Ja1LBD+yisY6RWM/BH7TJVXWsSjs2VwBSmvSX4HdBc= golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -484,12 +370,9 @@ golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -522,22 +405,11 @@ golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210220050731-9a76102bfb43/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210305230114-8fe3ee5dd75b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210320140829-1e4c9ba3b0c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210819135213-f52c844e1c1c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220204135822-1c1b9b1eba6a h1:ppl5mZgokTT8uPkmYOyEUmPTr3ypaKkg5eFOGrAmxxE= @@ -550,8 +422,6 @@ golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= @@ -564,7 +434,6 @@ golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3 golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= @@ -574,7 +443,6 @@ golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgw golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191112195655-aa38f8e97acc/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -597,19 +465,10 @@ golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200904185747-39188db58858/go.mod h1:Cj7w3i3Rnn0Xh82ur9kSqwfTHTeVxaDqrfMjpcNT6bE= -golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= -golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -631,12 +490,6 @@ google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0M google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= -google.golang.org/api v0.35.0/go.mod h1:/XrVsuzM0rZmrsbjJutiuftIzeuTQcEeaYcSk/mQ1dg= -google.golang.org/api v0.36.0/go.mod h1:+z5ficQTmoYpPn8LCUNVpK5I7hwkpjbcgqA7I34qYtE= -google.golang.org/api v0.40.0/go.mod h1:fYKFpnQN0DsDSKRVRcQSDQNtqWPfM9i+zNPxepjRCQ8= -google.golang.org/api v0.41.0/go.mod h1:RkxM5lITDfTzmyKFPt+wGrCJbVfniCr2ool8kTBzRTU= -google.golang.org/api v0.43.0/go.mod h1:nQsDGjRXMo4lvh5hP0TKqF244gqhGcr/YSIykhUk/94= -google.golang.org/api v0.44.0/go.mod h1:EBOGZqzyhtvMDoxwS97ctnh0zUmYY6CxqXsc1AvkYD8= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -668,24 +521,12 @@ google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfG google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200904004341-0bd0a958aa1d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201109203340-2640f1f9cdfb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201201144952-b05cb90ed32e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201210142538-e3217bee35cc/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201214200347-8c77b98c765d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210222152913-aa3ee6e6a81c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210303154014-9728d6b83eeb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210310155132-4ce2db91004e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210319143718-93e7006c17a6/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210402141018-6c239bbf2bb1/go.mod h1:9lPAdzaEmUacj36I+k7YKbEc5CXzPIeORRgDAUOu28A= -google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= @@ -698,14 +539,6 @@ google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKa google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.31.1/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= -google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA51WJ8= -google.golang.org/grpc v1.35.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.36.1/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -727,22 +560,18 @@ gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLF gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= -gopkg.in/ini.v1 v1.62.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/square/go-jose.v2 v2.4.1/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= From b0b7849be49d2a82460413eaf60e81b30eff7d6d Mon Sep 17 00:00:00 2001 From: Eugene R Date: Mon, 6 Feb 2023 10:24:28 +0200 Subject: [PATCH 011/348] Use pkg.go.dev badge in the readme (#947) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 28d0d1051f..89e90511f7 100644 --- a/README.md +++ b/README.md @@ -18,7 +18,7 @@ under the License. --> -[![GoDoc](https://img.shields.io/badge/Godoc-reference-blue.svg)](https://godoc.org/github.com/apache/pulsar-client-go) +[![PkgGoDev](https://pkg.go.dev/badge/github.com/apache/pulsar-client-go)](https://pkg.go.dev/github.com/apache/pulsar-client-go) [![Go Report Card](https://goreportcard.com/badge/github.com/apache/pulsar-client-go)](https://goreportcard.com/report/github.com/apache/pulsar-client-go) [![Language](https://img.shields.io/badge/Language-Go-blue.svg)](https://golang.org/) [![LICENSE](https://img.shields.io/hexpm/l/pulsar.svg)](/~https://github.com/apache/pulsar-client-go/blob/master/LICENSE) From 718a631aeafcbcd23912562698c4098b0ecd4411 Mon Sep 17 00:00:00 2001 From: Eugene R Date: Mon, 6 Feb 2023 10:25:02 +0200 Subject: [PATCH 012/348] Remove go1.11 code leftovers (#946) --- pulsar/internal/http_client.go | 2 +- pulsar/internal/http_client_go_1.11.go | 33 -------------------------- pulsar/internal/http_client_go_1.12.go | 27 --------------------- 3 files changed, 1 insertion(+), 61 deletions(-) delete mode 100644 pulsar/internal/http_client_go_1.11.go delete mode 100644 pulsar/internal/http_client_go_1.12.go diff --git a/pulsar/internal/http_client.go b/pulsar/internal/http_client.go index 672a65ed69..a7308cbe34 100644 --- a/pulsar/internal/http_client.go +++ b/pulsar/internal/http_client.go @@ -47,7 +47,7 @@ type httpClient struct { func (c *httpClient) Close() { if c.HTTPClient != nil { - CloseIdleConnections(c.HTTPClient) + c.HTTPClient.CloseIdleConnections() } } diff --git a/pulsar/internal/http_client_go_1.11.go b/pulsar/internal/http_client_go_1.11.go deleted file mode 100644 index 095a30fb36..0000000000 --- a/pulsar/internal/http_client_go_1.11.go +++ /dev/null @@ -1,33 +0,0 @@ -// 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. - -//go:build !go1.12 -// +build !go1.12 - -package internal - -import "net/http" - -func CloseIdleConnections(c *http.Client) { - type closeIdler interface { - CloseIdleConnections() - } - - if tr, ok := c.Transport.(closeIdler); ok { - tr.CloseIdleConnections() - } -} diff --git a/pulsar/internal/http_client_go_1.12.go b/pulsar/internal/http_client_go_1.12.go deleted file mode 100644 index 2d11d48b99..0000000000 --- a/pulsar/internal/http_client_go_1.12.go +++ /dev/null @@ -1,27 +0,0 @@ -// 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. - -//go:build go1.12 -// +build go1.12 - -package internal - -import "net/http" - -func CloseIdleConnections(c *http.Client) { - c.CloseIdleConnections() -} From fa4662c2c7a9cb3d8738c9ea616a799d2330ffd0 Mon Sep 17 00:00:00 2001 From: Eugene R Date: Tue, 7 Feb 2023 04:52:37 +0200 Subject: [PATCH 013/348] Migrate from the deprecated io/ioutil package (#942) --- .github/workflows/project.yml | 2 +- README.md | 2 +- integration-tests/license_test.go | 5 ++--- oauth2/authorization_tokenretriever_test.go | 4 ++-- oauth2/client_credentials_provider.go | 6 +++--- oauth2/client_credentials_provider_test.go | 3 +-- perf/pulsar-perf-go.go | 3 +-- pulsar/auth/athenz.go | 4 ++-- pulsar/auth/athenz_test.go | 4 ++-- pulsar/auth/basic_test.go | 4 ++-- pulsar/auth/oauth2_test.go | 3 +-- pulsar/auth/token.go | 4 ++-- pulsar/client_impl_test.go | 11 +++++------ pulsar/consumer_test.go | 4 ++-- pulsar/crypto/default_crypto_Key_reader.go | 4 ++-- pulsar/helper_for_test.go | 3 +-- pulsar/internal/compression/compression_bench_test.go | 8 ++++---- pulsar/internal/connection.go | 4 ++-- pulsar/internal/http_client.go | 8 ++++---- pulsar/internal/lookup_service_test.go | 6 +++--- 20 files changed, 43 insertions(+), 49 deletions(-) diff --git a/.github/workflows/project.yml b/.github/workflows/project.yml index dc5a755440..29e94d0c20 100644 --- a/.github/workflows/project.yml +++ b/.github/workflows/project.yml @@ -19,7 +19,7 @@ jobs: runs-on: ubuntu-latest strategy: matrix: - go-version: [1.15, 1.16, 1.17, 1.18, 1.19] + go-version: [1.16, 1.17, 1.18, 1.19] steps: - name: clean docker cache run: | diff --git a/README.md b/README.md index 89e90511f7..64700f8797 100644 --- a/README.md +++ b/README.md @@ -40,7 +40,7 @@ CGo based library. > **Note**: > -> While this library should work with Golang versions as early as 1.15, any bugs specific to versions earlier than 1.18 may not be fixed. +> While this library should work with Golang versions as early as 1.16, any bugs specific to versions earlier than 1.18 may not be fixed. ## Status diff --git a/integration-tests/license_test.go b/integration-tests/license_test.go index cd6d9d5ebc..ac89eb4dc2 100644 --- a/integration-tests/license_test.go +++ b/integration-tests/license_test.go @@ -18,7 +18,6 @@ package license_test import ( - "io/ioutil" "os" "path/filepath" "regexp" @@ -83,7 +82,7 @@ func TestLicense(t *testing.T) { switch filepath.Ext(path) { case ".go": - src, err := ioutil.ReadFile(path) + src, err := os.ReadFile(path) if err != nil { return nil } @@ -96,7 +95,7 @@ func TestLicense(t *testing.T) { case ".yaml": fallthrough case ".conf": - src, err := ioutil.ReadFile(path) + src, err := os.ReadFile(path) if err != nil { return nil } diff --git a/oauth2/authorization_tokenretriever_test.go b/oauth2/authorization_tokenretriever_test.go index fee573e0fd..6ae55d8fbd 100644 --- a/oauth2/authorization_tokenretriever_test.go +++ b/oauth2/authorization_tokenretriever_test.go @@ -21,7 +21,7 @@ import ( "bytes" "context" "encoding/json" - "io/ioutil" + "io" "net/http" "strings" "time" @@ -328,7 +328,7 @@ func buildResponse(statusCode int, body interface{}) *http.Response { resp := &http.Response{ StatusCode: statusCode, Header: map[string][]string{}, - Body: ioutil.NopCloser(bytes.NewReader(b)), + Body: io.NopCloser(bytes.NewReader(b)), } if strings.HasPrefix(string(b), "{") { resp.Header.Add("Content-Type", "application/json") diff --git a/oauth2/client_credentials_provider.go b/oauth2/client_credentials_provider.go index 5230ca3fee..3eb8ba44d4 100644 --- a/oauth2/client_credentials_provider.go +++ b/oauth2/client_credentials_provider.go @@ -19,7 +19,7 @@ package oauth2 import ( "encoding/json" - "io/ioutil" + "os" "strings" ) @@ -54,7 +54,7 @@ func (k *KeyFileProvider) GetClientCredentials() (*KeyFile, error) { switch { case strings.HasPrefix(k.KeyFile, FILE): filename := strings.TrimPrefix(k.KeyFile, FILE) - keyFile, err = ioutil.ReadFile(filename) + keyFile, err = os.ReadFile(filename) case strings.HasPrefix(k.KeyFile, DATA): keyFile = []byte(strings.TrimPrefix(k.KeyFile, DATA)) case strings.HasPrefix(k.KeyFile, "data:"): @@ -64,7 +64,7 @@ func (k *KeyFileProvider) GetClientCredentials() (*KeyFile, error) { } keyFile = url.Data default: - keyFile, err = ioutil.ReadFile(k.KeyFile) + keyFile, err = os.ReadFile(k.KeyFile) } if err != nil { return nil, err diff --git a/oauth2/client_credentials_provider_test.go b/oauth2/client_credentials_provider_test.go index f47967b6eb..869616d7b3 100644 --- a/oauth2/client_credentials_provider_test.go +++ b/oauth2/client_credentials_provider_test.go @@ -21,7 +21,6 @@ import ( "encoding/base64" "encoding/json" "fmt" - "io/ioutil" "os" "testing" @@ -45,7 +44,7 @@ func TestNewClientCredentialsProviderFromKeyFile(t *testing.T) { b, err := json.Marshal(keyFile) require.NoError(t, err) - tmpFile, err := ioutil.TempFile("", "key-file") + tmpFile, err := os.CreateTemp("", "key-file") require.NoError(t, err) defer func(name string) { _ = os.Remove(name) diff --git a/perf/pulsar-perf-go.go b/perf/pulsar-perf-go.go index aff0b73fa3..a672a30271 100644 --- a/perf/pulsar-perf-go.go +++ b/perf/pulsar-perf-go.go @@ -20,7 +20,6 @@ package main import ( "context" "fmt" - "io/ioutil" "net/http" _ "net/http/pprof" "os" @@ -57,7 +56,7 @@ func NewClient() (pulsar.Client, error) { if clientArgs.TokenFile != "" { // read JWT from the file - tokenBytes, err := ioutil.ReadFile(clientArgs.TokenFile) + tokenBytes, err := os.ReadFile(clientArgs.TokenFile) if err != nil { log.WithError(err).Errorf("failed to read Pulsar JWT from a file %s", clientArgs.TokenFile) os.Exit(1) diff --git a/pulsar/auth/athenz.go b/pulsar/auth/athenz.go index 9d17ac0394..f490be95ac 100644 --- a/pulsar/auth/athenz.go +++ b/pulsar/auth/athenz.go @@ -21,8 +21,8 @@ import ( "crypto/tls" "encoding/base64" "errors" - "io/ioutil" "net/http" + "os" "regexp" "strings" "time" @@ -117,7 +117,7 @@ func (p *athenzAuthProvider) Init() error { } keyData = key } else if uriSt.Scheme == "file" { - key, err := ioutil.ReadFile(uriSt.Path) + key, err := os.ReadFile(uriSt.Path) if err != nil { return err } diff --git a/pulsar/auth/athenz_test.go b/pulsar/auth/athenz_test.go index 97cc0aee19..8541ca98f5 100644 --- a/pulsar/auth/athenz_test.go +++ b/pulsar/auth/athenz_test.go @@ -20,7 +20,7 @@ package auth import ( "bytes" "errors" - "io/ioutil" + "os" "testing" "time" @@ -69,7 +69,7 @@ func (m *MockRoleToken) RoleTokenValue() (string, error) { func MockZmsNewTokenBuilder(domain, name string, privateKeyPEM []byte, keyVersion string) (zms.TokenBuilder, error) { // assertion - key, err := ioutil.ReadFile(tlsClientKeyPath) + key, err := os.ReadFile(tlsClientKeyPath) if err != nil { return nil, err } diff --git a/pulsar/auth/basic_test.go b/pulsar/auth/basic_test.go index b212d0ba5d..ead7f3dc23 100644 --- a/pulsar/auth/basic_test.go +++ b/pulsar/auth/basic_test.go @@ -19,7 +19,7 @@ package auth import ( "errors" - "io/ioutil" + "io" "net/http" "net/http/httptest" "testing" @@ -51,7 +51,7 @@ func TestNewAuthenticationBasicWithParams(t *testing.T) { resp, err := client.Get(s.URL) require.NoError(t, err) - body, err := ioutil.ReadAll(resp.Body) + body, err := io.ReadAll(resp.Body) _ = resp.Body.Close() require.NoError(t, err) require.Equal(t, []byte("Basic YWRtaW46MTIzNDU2"), body) diff --git a/pulsar/auth/oauth2_test.go b/pulsar/auth/oauth2_test.go index 536c0ddccf..86d364044d 100644 --- a/pulsar/auth/oauth2_test.go +++ b/pulsar/auth/oauth2_test.go @@ -19,7 +19,6 @@ package auth import ( "fmt" - "io/ioutil" "net/http" "net/http/httptest" "os" @@ -64,7 +63,7 @@ func mockKeyFile(server string) (string, error) { if err != nil { return "", err } - kf, err := ioutil.TempFile(pwd, "test_oauth2") + kf, err := os.CreateTemp(pwd, "test_oauth2") if err != nil { return "", err } diff --git a/pulsar/auth/token.go b/pulsar/auth/token.go index b5af86b935..898b6537fd 100644 --- a/pulsar/auth/token.go +++ b/pulsar/auth/token.go @@ -20,8 +20,8 @@ package auth import ( "crypto/tls" "fmt" - "io/ioutil" "net/http" + "os" "strings" "github.com/pkg/errors" @@ -69,7 +69,7 @@ func NewAuthenticationTokenFromSupplier(tokenSupplier func() (string, error)) Pr func NewAuthenticationTokenFromFile(tokenFilePath string) Provider { return &tokenAuthProvider{ tokenSupplier: func() (string, error) { - data, err := ioutil.ReadFile(tokenFilePath) + data, err := os.ReadFile(tokenFilePath) if err != nil { return "", err } diff --git a/pulsar/client_impl_test.go b/pulsar/client_impl_test.go index 6ead0575bd..299203f091 100644 --- a/pulsar/client_impl_test.go +++ b/pulsar/client_impl_test.go @@ -21,7 +21,6 @@ import ( "context" "crypto/tls" "fmt" - "io/ioutil" "net/http" "net/http/httptest" "os" @@ -189,7 +188,7 @@ func TestTLSAuthWithCertSupplier(t *testing.T) { } func TestTokenAuth(t *testing.T) { - token, err := ioutil.ReadFile(tokenFilePath) + token, err := os.ReadFile(tokenFilePath) assert.NoError(t, err) client, err := NewClient(ClientOptions{ @@ -212,7 +211,7 @@ func TestTokenAuthWithSupplier(t *testing.T) { client, err := NewClient(ClientOptions{ URL: serviceURL, Authentication: NewAuthenticationTokenFromSupplier(func() (s string, err error) { - token, err := ioutil.ReadFile(tokenFilePath) + token, err := os.ReadFile(tokenFilePath) if err != nil { return "", err } @@ -287,7 +286,7 @@ func mockKeyFile(server string) (string, error) { if err != nil { return "", err } - kf, err := ioutil.TempFile(pwd, "test_oauth2") + kf, err := os.CreateTemp(pwd, "test_oauth2") if err != nil { return "", err } @@ -838,7 +837,7 @@ func TestHTTPSAuthWithCertSupplier(t *testing.T) { } func TestHTTPTokenAuth(t *testing.T) { - token, err := ioutil.ReadFile(tokenFilePath) + token, err := os.ReadFile(tokenFilePath) assert.NoError(t, err) client, err := NewClient(ClientOptions{ @@ -861,7 +860,7 @@ func TestHTTPTokenAuthWithSupplier(t *testing.T) { client, err := NewClient(ClientOptions{ URL: webServiceURL, Authentication: NewAuthenticationTokenFromSupplier(func() (s string, err error) { - token, err := ioutil.ReadFile(tokenFilePath) + token, err := os.ReadFile(tokenFilePath) if err != nil { return "", err } diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 366be5de14..0eb7aae572 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -21,9 +21,9 @@ import ( "context" "errors" "fmt" - "io/ioutil" "log" "net/http" + "os" "strconv" "sync" "sync/atomic" @@ -3422,7 +3422,7 @@ func (d *EncKeyReader) PrivateKey(keyName string, keyMeta map[string]string) (*c } func readKey(keyName, path string, keyMeta map[string]string) (*crypto.EncryptionKeyInfo, error) { - key, err := ioutil.ReadFile(path) + key, err := os.ReadFile(path) if err != nil { return nil, err } diff --git a/pulsar/crypto/default_crypto_Key_reader.go b/pulsar/crypto/default_crypto_Key_reader.go index 6378d4e9ae..8dfb2bc039 100644 --- a/pulsar/crypto/default_crypto_Key_reader.go +++ b/pulsar/crypto/default_crypto_Key_reader.go @@ -17,7 +17,7 @@ package crypto -import "io/ioutil" +import "os" // FileKeyReader default implementation of KeyReader type FileKeyReader struct { @@ -43,7 +43,7 @@ func (d *FileKeyReader) PrivateKey(keyName string, keyMeta map[string]string) (* } func readKey(keyName, path string, keyMeta map[string]string) (*EncryptionKeyInfo, error) { - key, err := ioutil.ReadFile(path) + key, err := os.ReadFile(path) if err != nil { return nil, err } diff --git a/pulsar/helper_for_test.go b/pulsar/helper_for_test.go index d6a4f0042d..a2888d0d58 100644 --- a/pulsar/helper_for_test.go +++ b/pulsar/helper_for_test.go @@ -22,7 +22,6 @@ import ( "encoding/json" "fmt" "io" - "io/ioutil" "net/http" "path" "strings" @@ -110,7 +109,7 @@ func httpDo(method string, requestPath string, in interface{}, out interface{}) } if out != nil { - outBytes, err := ioutil.ReadAll(resp.Body) + outBytes, err := io.ReadAll(resp.Body) if err != nil { return err } diff --git a/pulsar/internal/compression/compression_bench_test.go b/pulsar/internal/compression/compression_bench_test.go index de0ca48d5d..f615ff066a 100644 --- a/pulsar/internal/compression/compression_bench_test.go +++ b/pulsar/internal/compression/compression_bench_test.go @@ -18,7 +18,7 @@ package compression import ( - "io/ioutil" + "os" "testing" ) @@ -27,7 +27,7 @@ const ( ) func testCompression(b *testing.B, provider Provider) { - data, err := ioutil.ReadFile(dataSampleFile) + data, err := os.ReadFile(dataSampleFile) if err != nil { b.Error(err) } @@ -45,7 +45,7 @@ func testCompression(b *testing.B, provider Provider) { func testDecompression(b *testing.B, provider Provider) { // Read data sample file - data, err := ioutil.ReadFile(dataSampleFile) + data, err := os.ReadFile(dataSampleFile) if err != nil { b.Error(err) } @@ -97,7 +97,7 @@ func BenchmarkDecompression(b *testing.B) { func BenchmarkCompressionParallel(b *testing.B) { b.ReportAllocs() - data, err := ioutil.ReadFile(dataSampleFile) + data, err := os.ReadFile(dataSampleFile) if err != nil { b.Error(err) } diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index be95240b36..cb4af335ed 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -22,9 +22,9 @@ import ( "crypto/x509" "errors" "fmt" - "io/ioutil" "net" "net/url" + "os" "sync" "sync/atomic" "time" @@ -979,7 +979,7 @@ func (c *connection) getTLSConfig() (*tls.Config, error) { } if c.tlsOptions.TrustCertsFilePath != "" { - caCerts, err := ioutil.ReadFile(c.tlsOptions.TrustCertsFilePath) + caCerts, err := os.ReadFile(c.tlsOptions.TrustCertsFilePath) if err != nil { return nil, err } diff --git a/pulsar/internal/http_client.go b/pulsar/internal/http_client.go index a7308cbe34..dccc1431b3 100644 --- a/pulsar/internal/http_client.go +++ b/pulsar/internal/http_client.go @@ -23,9 +23,9 @@ import ( "crypto/x509" "encoding/json" "io" - "io/ioutil" "net/http" "net/url" + "os" "path" "time" @@ -209,7 +209,7 @@ func (c *httpClient) GetWithOptions(endpoint string, obj interface{}, params map return nil, err } } else { - body, err := ioutil.ReadAll(resp.Body) + body, err := io.ReadAll(resp.Body) if err != nil { return nil, err } @@ -311,7 +311,7 @@ func safeRespClose(resp *http.Response) { // responseError is used to parse a response into a client error func responseError(resp *http.Response) error { var e error - body, err := ioutil.ReadAll(resp.Body) + body, err := io.ReadAll(resp.Body) reason := "" code := resp.StatusCode if err != nil { @@ -338,7 +338,7 @@ func getDefaultTransport(tlsConfig *TLSOptions) (http.RoundTripper, error) { InsecureSkipVerify: tlsConfig.AllowInsecureConnection, } if len(tlsConfig.TrustCertsFilePath) > 0 { - rootCA, err := ioutil.ReadFile(tlsConfig.TrustCertsFilePath) + rootCA, err := os.ReadFile(tlsConfig.TrustCertsFilePath) if err != nil { return nil, err } diff --git a/pulsar/internal/lookup_service_test.go b/pulsar/internal/lookup_service_test.go index 9c55e9f142..1504f3f2bb 100644 --- a/pulsar/internal/lookup_service_test.go +++ b/pulsar/internal/lookup_service_test.go @@ -20,7 +20,7 @@ package internal import ( "bytes" "encoding/json" - "io/ioutil" + "io" "net/url" "strings" "testing" @@ -574,7 +574,7 @@ func mockHTTPGetLookupResult(obj interface{}) error { "httpUrl": "http://broker-1:8080", "httpUrlTls": "" }` - r := ioutil.NopCloser(bytes.NewReader([]byte(jsonResponse))) + r := io.NopCloser(bytes.NewReader([]byte(jsonResponse))) dec := json.NewDecoder(r) err := dec.Decode(obj) return err @@ -584,7 +584,7 @@ func mockHTTPGetPartitionedTopicMetadataResult(obj interface{}) error { jsonResponse := `{ "partitions": 1 }` - r := ioutil.NopCloser(bytes.NewReader([]byte(jsonResponse))) + r := io.NopCloser(bytes.NewReader([]byte(jsonResponse))) dec := json.NewDecoder(r) err := dec.Decode(obj) return err From c346caf0b647650196bf120067459e27010e6522 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 8 Feb 2023 16:35:29 +0800 Subject: [PATCH 014/348] Improve test script (#951) --- .github/workflows/project.yml | 2 ++ Dockerfile | 2 ++ Makefile | 3 +-- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/.github/workflows/project.yml b/.github/workflows/project.yml index 29e94d0c20..a52a4be4f6 100644 --- a/.github/workflows/project.yml +++ b/.github/workflows/project.yml @@ -14,6 +14,8 @@ jobs: steps: - uses: actions/checkout@v3 - uses: golangci/golangci-lint-action@v3 + with: + version: v1.50.1 integration-tests: runs-on: ubuntu-latest diff --git a/Dockerfile b/Dockerfile index dcc6a26f7b..f66eba54b5 100644 --- a/Dockerfile +++ b/Dockerfile @@ -37,4 +37,6 @@ COPY integration-tests/conf/.htpasswd \ integration-tests/conf/standalone.conf \ /pulsar/conf/ +COPY . /pulsar/pulsar-client-go + ENV PULSAR_EXTRA_OPTS="-Dpulsar.auth.basic.conf=/pulsar/conf/.htpasswd" diff --git a/Makefile b/Makefile index 130057ec90..3245434736 100644 --- a/Makefile +++ b/Makefile @@ -35,8 +35,7 @@ container: --build-arg PULSAR_IMAGE="${PULSAR_IMAGE}" . test: container - docker run -i -v ${PWD}:/pulsar-client-go ${IMAGE_NAME} \ - bash -c "cd /pulsar-client-go && ./scripts/run-ci.sh" + docker run -i ${IMAGE_NAME} bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci.sh" clean: docker rmi --force $(IMAGE_NAME) || true From 93a5a765b9ff9d756d0adc5d2de30f31e4cbf29f Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 9 Feb 2023 11:04:39 +0800 Subject: [PATCH 015/348] [feat] Support Exclusive Producer access mode. (#944) * [feat] Support Exclusive Producer access mode. * Use uint ptr instead int64 --- pulsar/producer.go | 17 +++++++++++++++++ pulsar/producer_partition.go | 19 +++++++++++++++++++ pulsar/producer_test.go | 32 ++++++++++++++++++++++++++++++++ 3 files changed, 68 insertions(+) diff --git a/pulsar/producer.go b/pulsar/producer.go index d088fb2d60..d9ac34bd77 100644 --- a/pulsar/producer.go +++ b/pulsar/producer.go @@ -55,6 +55,17 @@ const ( Better ) +type ProducerAccessMode int + +const ( + // ProducerAccessModeShared is default multiple producers can publish on a topic. + ProducerAccessModeShared ProducerAccessMode = iota + + // ProducerAccessModeExclusive is required exclusive access for producer. + // Fail immediately if there's already a producer connected. + ProducerAccessModeExclusive +) + // TopicMetadata represents a topic metadata. type TopicMetadata interface { // NumPartitions returns the number of partitions for a particular topic. @@ -187,6 +198,12 @@ type ProducerOptions struct { // ChunkMaxMessageSize is the max size of single chunk payload. // It will actually only take effect if it is smaller than the maxMessageSize from the broker. ChunkMaxMessageSize uint + + // The type of access to the topic that the producer requires. (default ProducerAccessModeShared) + // Options: + // - ProducerAccessModeShared + // - ProducerAccessModeExclusive + ProducerAccessMode } // Producer is used to publish messages on a topic diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index b0467f5409..eece055b4c 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -95,6 +95,7 @@ type partitionProducer struct { metrics *internal.LeveledMetrics epoch uint64 schemaCache *schemaCache + topicEpoch *uint64 } type schemaCache struct { @@ -237,6 +238,11 @@ func (p *partitionProducer) grabCnx() error { Schema: pbSchema, Epoch: proto.Uint64(atomic.LoadUint64(&p.epoch)), UserProvidedProducerName: proto.Bool(p.userProvidedProducerName), + ProducerAccessMode: toProtoProducerAccessMode(p.options.ProducerAccessMode).Enum(), + } + + if p.topicEpoch != nil { + cmdProducer.TopicEpoch = proto.Uint64(*p.topicEpoch) } if p.producerName != "" { @@ -253,6 +259,8 @@ func (p *partitionProducer) grabCnx() error { } p.producerName = res.Response.ProducerSuccess.GetProducerName() + nextTopicEpoch := res.Response.ProducerSuccess.GetTopicEpoch() + p.topicEpoch = &nextTopicEpoch if p.options.Encryption != nil { p.encryptor = internalcrypto.NewProducerEncryptor(p.options.Encryption.Keys, @@ -1352,3 +1360,14 @@ func (c *chunkRecorder) setFirstChunkID(msgID messageID) { func (c *chunkRecorder) setLastChunkID(msgID messageID) { c.chunkedMsgID.messageID = msgID } + +func toProtoProducerAccessMode(accessMode ProducerAccessMode) pb.ProducerAccessMode { + switch accessMode { + case ProducerAccessModeShared: + return pb.ProducerAccessMode_Shared + case ProducerAccessModeExclusive: + return pb.ProducerAccessMode_Exclusive + } + + return pb.ProducerAccessMode_Shared +} diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 75fc6db884..d7950ebeb5 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -23,6 +23,7 @@ import ( "fmt" "net/http" "strconv" + "strings" "sync" "sync/atomic" "testing" @@ -1705,3 +1706,34 @@ func TestProducerWithSchemaAndConsumerSchemaNotFound(t *testing.T) { // should fail with error but not panic assert.Error(t, err) } + +func TestExclusiveProducer(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: serviceURL, + }) + assert.NoError(t, err) + defer client.Close() + + topicName := newTopicName() + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topicName, + ProducerAccessMode: ProducerAccessModeExclusive, + }) + + assert.NoError(t, err) + assert.NotNil(t, producer) + defer producer.Close() + + _, err = client.CreateProducer(ProducerOptions{ + Topic: topicName, + ProducerAccessMode: ProducerAccessModeExclusive, + }) + assert.Error(t, err, "Producer should be fenced") + assert.True(t, strings.Contains(err.Error(), "ProducerFenced")) + + _, err = client.CreateProducer(ProducerOptions{ + Topic: topicName, + }) + assert.Error(t, err, "Producer should be failed") + assert.True(t, strings.Contains(err.Error(), "ProducerBusy")) +} From cf3207f4637d80efbcbc0f7646ea326d9f9bac6b Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 20 Feb 2023 22:32:39 +0800 Subject: [PATCH 016/348] [feat] Support WaitForExclusive producer access mode. (#958) * [feat] Support WaitForExclusive producer access mode. * Remove useless defer. --- pulsar/internal/connection.go | 17 ++++++++++++-- pulsar/internal/rpc_client.go | 22 +++++++++++++----- pulsar/producer.go | 3 +++ pulsar/producer_partition.go | 2 ++ pulsar/producer_test.go | 42 +++++++++++++++++++++++++++++++++++ 5 files changed, 78 insertions(+), 8 deletions(-) diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index cb4af335ed..67b6f32b43 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -523,8 +523,14 @@ func (c *connection) internalReceivedCommand(cmd *pb.BaseCommand, headersAndPayl c.handleResponse(cmd.Success.GetRequestId(), cmd) case pb.BaseCommand_PRODUCER_SUCCESS: - c.handleResponse(cmd.ProducerSuccess.GetRequestId(), cmd) - + if !*cmd.ProducerSuccess.ProducerReady { + request, ok := c.findPendingRequest(cmd.ProducerSuccess.GetRequestId()) + if ok { + request.callback(cmd, nil) + } + } else { + c.handleResponse(cmd.ProducerSuccess.GetRequestId(), cmd) + } case pb.BaseCommand_PARTITIONED_METADATA_RESPONSE: c.checkServerError(cmd.PartitionMetadataResponse.Error) c.handleResponse(cmd.PartitionMetadataResponse.GetRequestId(), cmd) @@ -748,6 +754,13 @@ func (c *connection) deletePendingRequest(requestID uint64) (*request, bool) { return request, ok } +func (c *connection) findPendingRequest(requestID uint64) (*request, bool) { + c.pendingLock.Lock() + defer c.pendingLock.Unlock() + request, ok := c.pendingReqs[requestID] + return request, ok +} + func (c *connection) failPendingRequests(err error) bool { c.pendingLock.Lock() defer c.pendingLock.Unlock() diff --git a/pulsar/internal/rpc_client.go b/pulsar/internal/rpc_client.go index 378ab4ff7f..0ee8ca93c7 100644 --- a/pulsar/internal/rpc_client.go +++ b/pulsar/internal/rpc_client.go @@ -129,14 +129,24 @@ func (c *rpcClient) Request(logicalAddr *url.URL, physicalAddr *url.URL, request Cnx: cnx, Response: response, }, err} - close(ch) }) - select { - case res := <-ch: - return res.RPCResult, res.error - case <-time.After(c.requestTimeout): - return nil, ErrRequestTimeOut + timeoutCh := time.After(c.requestTimeout) + for { + select { + case res := <-ch: + // Ignoring producer not ready response. + // Continue to wait for the producer to create successfully + if res.error == nil && *res.RPCResult.Response.Type == pb.BaseCommand_PRODUCER_SUCCESS { + if !*res.RPCResult.Response.ProducerSuccess.ProducerReady { + timeoutCh = nil + break + } + } + return res.RPCResult, res.error + case <-timeoutCh: + return nil, ErrRequestTimeOut + } } } diff --git a/pulsar/producer.go b/pulsar/producer.go index d9ac34bd77..8fcb891eb8 100644 --- a/pulsar/producer.go +++ b/pulsar/producer.go @@ -64,6 +64,9 @@ const ( // ProducerAccessModeExclusive is required exclusive access for producer. // Fail immediately if there's already a producer connected. ProducerAccessModeExclusive + + // ProducerAccessModeWaitForExclusive is pending until producer can acquire exclusive access. + ProducerAccessModeWaitForExclusive ) // TopicMetadata represents a topic metadata. diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index eece055b4c..160693cbce 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1367,6 +1367,8 @@ func toProtoProducerAccessMode(accessMode ProducerAccessMode) pb.ProducerAccessM return pb.ProducerAccessMode_Shared case ProducerAccessModeExclusive: return pb.ProducerAccessMode_Exclusive + case ProducerAccessModeWaitForExclusive: + return pb.ProducerAccessMode_WaitForExclusive } return pb.ProducerAccessMode_Shared diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index d7950ebeb5..e69a14ce71 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -1737,3 +1737,45 @@ func TestExclusiveProducer(t *testing.T) { assert.Error(t, err, "Producer should be failed") assert.True(t, strings.Contains(err.Error(), "ProducerBusy")) } + +func TestWaitForExclusiveProducer(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: serviceURL, + // Set the request timeout is 200ms + OperationTimeout: 200 * time.Millisecond, + }) + assert.NoError(t, err) + defer client.Close() + + topicName := newTopicName() + producer1, err := client.CreateProducer(ProducerOptions{ + Topic: topicName, + ProducerAccessMode: ProducerAccessModeExclusive, + }) + assert.NoError(t, err) + assert.NotNil(t, producer1) + + wg := sync.WaitGroup{} + wg.Add(1) + go func() { + producer2, err := client.CreateProducer(ProducerOptions{ + Topic: topicName, + ProducerAccessMode: ProducerAccessModeWaitForExclusive, + }) + defer producer2.Close() + assert.NoError(t, err) + assert.NotNil(t, producer2) + + id, err := producer2.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }) + assert.Nil(t, err) + assert.NotNil(t, id) + wg.Done() + }() + // Because set the request timeout is 200ms before. + // Here waite 300ms to cover wait for exclusive producer never timeout + time.Sleep(300 * time.Millisecond) + producer1.Close() + wg.Wait() +} From 7d257b01b1216e0009495a4aaf83b1716ba458a9 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 22 Feb 2023 08:38:22 +0800 Subject: [PATCH 017/348] Support grouping ACK requests by time and size (#957) * Support grouping ACK requests by time and size Fixes #949 ### Motivation Currently the Go client does not support grouping ACK requests, so each time `Ack` (or similar APIs) is called, a ACK request will be sent, which could downgrade the performance. We need to support configuring the time and size to cache `MessageID` before sending ACK requests. ### Modifications - Add an `AckGroupingOptions` field to `ConsumerOptions`, when it's nil, use 100ms as the max time and 1000 as the max size. - Add an `ackGroupingTracker` interface to support grouping ACK requests - When `AckWithResponse` is false, adding the `MessageID` instance to the tracker instead of sending the requests to `eventsCh`. ### Verifying this change - [ ] Make sure that the change passes the CI checks. This change added tests and can be verified as follows: - Added `ack_grouping_tracker_test.go` to verify `ackGroupingTracker` in various cases - The consumer side change can be covered by existing tests because the default `AckGroupingOptions` config is `{ MaxSize: 1000, MaxTime: 100*time.Millisecond }`. * Fix flushAndClean race * Use unbuffered channel for flush operations * Apply different AckGroupingOptions and expose this config --- pulsar/ack_grouping_tracker.go | 298 ++++++++++++++++++++++++++++ pulsar/ack_grouping_tracker_test.go | 197 ++++++++++++++++++ pulsar/consumer.go | 21 ++ pulsar/consumer_impl.go | 1 + pulsar/consumer_partition.go | 83 +++++--- pulsar/consumer_partition_test.go | 6 + pulsar/consumer_test.go | 54 ++--- pulsar/message.go | 39 ++++ 8 files changed, 647 insertions(+), 52 deletions(-) create mode 100644 pulsar/ack_grouping_tracker.go create mode 100644 pulsar/ack_grouping_tracker_test.go diff --git a/pulsar/ack_grouping_tracker.go b/pulsar/ack_grouping_tracker.go new file mode 100644 index 0000000000..dbc70f5d70 --- /dev/null +++ b/pulsar/ack_grouping_tracker.go @@ -0,0 +1,298 @@ +// 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 pulsar + +import ( + "time" + + "github.com/bits-and-blooms/bitset" +) + +type ackGroupingTracker interface { + add(id MessageID) + + addCumulative(id MessageID) + + isDuplicate(id MessageID) bool + + flush() + + flushAndClean() + + close() +} + +type ackFlushType int + +const ( + flushOnly ackFlushType = iota + flushAndClean + flushAndClose +) + +func newAckGroupingTracker(options *AckGroupingOptions, + ackIndividual func(id MessageID), + ackCumulative func(id MessageID)) ackGroupingTracker { + if options == nil { + options = &AckGroupingOptions{ + MaxSize: 1000, + MaxTime: 100 * time.Millisecond, + } + } + + if options.MaxSize <= 1 { + return &immediateAckGroupingTracker{ + ackIndividual: ackIndividual, + ackCumulative: ackCumulative, + } + } + + c := &cachedAcks{ + singleAcks: make([]MessageID, options.MaxSize), + pendingAcks: make(map[int64]*bitset.BitSet), + lastCumulativeAck: EarliestMessageID(), + ackIndividual: ackIndividual, + ackCumulative: ackCumulative, + ackList: func(ids []MessageID) { + // TODO: support ack a list of MessageIDs + for _, id := range ids { + ackIndividual(id) + } + }, + } + + timeout := time.NewTicker(time.Hour) + if options.MaxTime > 0 { + timeout = time.NewTicker(options.MaxTime) + } else { + timeout.Stop() + } + t := &timedAckGroupingTracker{ + ackIndividualCh: make(chan MessageID), + ackCumulativeCh: make(chan MessageID), + duplicateIDCh: make(chan MessageID), + duplicateResultCh: make(chan bool), + flushCh: make(chan ackFlushType), + waitFlushCh: make(chan bool), + } + go func() { + for { + select { + case id := <-t.ackIndividualCh: + if c.addAndCheckIfFull(id) { + c.flushIndividualAcks() + if options.MaxTime > 0 { + timeout.Reset(options.MaxTime) + } + } + case id := <-t.ackCumulativeCh: + c.tryUpdateLastCumulativeAck(id) + if options.MaxTime <= 0 { + c.flushCumulativeAck() + } + case id := <-t.duplicateIDCh: + t.duplicateResultCh <- c.isDuplicate(id) + case <-timeout.C: + c.flush() + case ackFlushType := <-t.flushCh: + timeout.Stop() + c.flush() + if ackFlushType == flushAndClean { + c.clean() + } + t.waitFlushCh <- true + if ackFlushType == flushAndClose { + return + } + } + } + }() + return t +} + +type immediateAckGroupingTracker struct { + ackIndividual func(id MessageID) + ackCumulative func(id MessageID) +} + +func (i *immediateAckGroupingTracker) add(id MessageID) { + i.ackIndividual(id) +} + +func (i *immediateAckGroupingTracker) addCumulative(id MessageID) { + i.ackCumulative(id) +} + +func (i *immediateAckGroupingTracker) isDuplicate(id MessageID) bool { + return false +} + +func (i *immediateAckGroupingTracker) flush() { +} + +func (i *immediateAckGroupingTracker) flushAndClean() { +} + +func (i *immediateAckGroupingTracker) close() { +} + +type cachedAcks struct { + singleAcks []MessageID + index int + + // Key is the hash code of the ledger id and the netry id, + // Value is the bit set that represents which messages are acknowledged if the entry stores a batch. + // The bit 1 represents the message has been acknowledged, i.e. the bits "111" represents all messages + // in the batch whose batch size is 3 are not acknowledged. + // After the 1st message (i.e. batch index is 0) is acknowledged, the bits will become "011". + // Value is nil if the entry represents a single message. + pendingAcks map[int64]*bitset.BitSet + + lastCumulativeAck MessageID + cumulativeAckRequired bool + + ackIndividual func(id MessageID) + ackCumulative func(id MessageID) + ackList func(ids []MessageID) +} + +func (t *cachedAcks) addAndCheckIfFull(id MessageID) bool { + t.singleAcks[t.index] = id + t.index++ + key := messageIDHash(id) + ackSet, found := t.pendingAcks[key] + if !found { + if messageIDIsBatch(id) { + ackSet = bitset.New(uint(id.BatchSize())) + for i := 0; i < int(id.BatchSize()); i++ { + ackSet.Set(uint(i)) + } + t.pendingAcks[key] = ackSet + } else { + t.pendingAcks[key] = nil + } + } + if ackSet != nil { + ackSet.Clear(uint(id.BatchIdx())) + } + return t.index == len(t.singleAcks) +} + +func (t *cachedAcks) tryUpdateLastCumulativeAck(id MessageID) { + if messageIDCompare(t.lastCumulativeAck, id) < 0 { + t.lastCumulativeAck = id + t.cumulativeAckRequired = true + } +} + +func (t *cachedAcks) isDuplicate(id MessageID) bool { + if messageIDCompare(t.lastCumulativeAck, id) >= 0 { + return true + } + ackSet, found := t.pendingAcks[messageIDHash(id)] + if !found { + return false + } + if ackSet == nil || !messageIDIsBatch(id) { + // NOTE: should we panic when ackSet != nil and messageIDIsBatch(id) is true? + return true + } + // 0 represents the message has been acknowledged + return !ackSet.Test(uint(id.BatchIdx())) +} + +func (t *cachedAcks) flushIndividualAcks() { + if t.index > 0 { + t.ackList(t.singleAcks[0:t.index]) + for _, id := range t.singleAcks[0:t.index] { + key := messageIDHash(id) + ackSet, found := t.pendingAcks[key] + if !found { + continue + } + if ackSet == nil { + delete(t.pendingAcks, key) + } else { + ackSet.Clear(uint(id.BatchIdx())) + if ackSet.None() { // all messages have been acknowledged + delete(t.pendingAcks, key) + } + } + delete(t.pendingAcks, messageIDHash(id)) + } + t.index = 0 + } +} + +func (t *cachedAcks) flushCumulativeAck() { + if t.cumulativeAckRequired { + t.ackCumulative(t.lastCumulativeAck) + t.cumulativeAckRequired = false + } +} + +func (t *cachedAcks) flush() { + t.flushIndividualAcks() + t.flushCumulativeAck() +} + +func (t *cachedAcks) clean() { + maxSize := len(t.singleAcks) + t.singleAcks = make([]MessageID, maxSize) + t.index = 0 + t.pendingAcks = make(map[int64]*bitset.BitSet) + t.lastCumulativeAck = EarliestMessageID() + t.cumulativeAckRequired = false +} + +type timedAckGroupingTracker struct { + ackIndividualCh chan MessageID + ackCumulativeCh chan MessageID + duplicateIDCh chan MessageID + duplicateResultCh chan bool + flushCh chan ackFlushType + waitFlushCh chan bool +} + +func (t *timedAckGroupingTracker) add(id MessageID) { + t.ackIndividualCh <- id +} + +func (t *timedAckGroupingTracker) addCumulative(id MessageID) { + t.ackCumulativeCh <- id +} + +func (t *timedAckGroupingTracker) isDuplicate(id MessageID) bool { + t.duplicateIDCh <- id + return <-t.duplicateResultCh +} + +func (t *timedAckGroupingTracker) flush() { + t.flushCh <- flushOnly + <-t.waitFlushCh +} + +func (t *timedAckGroupingTracker) flushAndClean() { + t.flushCh <- flushAndClean + <-t.waitFlushCh +} + +func (t *timedAckGroupingTracker) close() { + t.flushCh <- flushAndClose + <-t.waitFlushCh +} diff --git a/pulsar/ack_grouping_tracker_test.go b/pulsar/ack_grouping_tracker_test.go new file mode 100644 index 0000000000..d7903e8f93 --- /dev/null +++ b/pulsar/ack_grouping_tracker_test.go @@ -0,0 +1,197 @@ +// 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 pulsar + +import ( + "fmt" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestNoCacheTracker(t *testing.T) { + tests := []AckGroupingOptions{ + { + MaxSize: 0, + MaxTime: 10 * time.Hour, + }, + { + MaxSize: 1, + MaxTime: 10 * time.Hour, + }, + } + for _, option := range tests { + t.Run(fmt.Sprintf("TestAckImmediately_size_%v_time_%vs", option.MaxSize, option.MaxTime.Seconds()), + func(t *testing.T) { + ledgerID0 := int64(-1) + ledgerID1 := int64(-1) + tracker := newAckGroupingTracker(&option, + func(id MessageID) { ledgerID0 = id.LedgerID() }, + func(id MessageID) { ledgerID1 = id.LedgerID() }) + + tracker.add(&messageID{ledgerID: 1}) + assert.Equal(t, atomic.LoadInt64(&ledgerID0), int64(1)) + tracker.addCumulative(&messageID{ledgerID: 2}) + assert.Equal(t, atomic.LoadInt64(&ledgerID1), int64(2)) + }) + } +} + +type mockAcker struct { + sync.Mutex + ledgerIDs []int64 + cumulativeLedgerID int64 +} + +func (a *mockAcker) ack(id MessageID) { + defer a.Unlock() + a.Lock() + a.ledgerIDs = append(a.ledgerIDs, id.LedgerID()) +} + +func (a *mockAcker) ackCumulative(id MessageID) { + atomic.StoreInt64(&a.cumulativeLedgerID, id.LedgerID()) +} + +func (a *mockAcker) getLedgerIDs() []int64 { + defer a.Unlock() + a.Lock() + return a.ledgerIDs +} + +func (a *mockAcker) getCumulativeLedgerID() int64 { + return atomic.LoadInt64(&a.cumulativeLedgerID) +} + +func (a *mockAcker) reset() { + a.ledgerIDs = make([]int64, 0) + a.cumulativeLedgerID = int64(0) +} + +func TestCachedTracker(t *testing.T) { + var acker mockAcker + tracker := newAckGroupingTracker(&AckGroupingOptions{MaxSize: 3, MaxTime: 0}, + func(id MessageID) { acker.ack(id) }, func(id MessageID) { acker.ackCumulative(id) }) + + tracker.add(&messageID{ledgerID: 1}) + tracker.add(&messageID{ledgerID: 2}) + for i := 1; i <= 2; i++ { + assert.True(t, tracker.isDuplicate(&messageID{ledgerID: int64(i)})) + } + assert.Equal(t, 0, len(acker.getLedgerIDs())) + tracker.add(&messageID{ledgerID: 3}) + assert.Eventually(t, func() bool { return len(acker.getLedgerIDs()) > 0 }, + 10*time.Millisecond, 2*time.Millisecond) + assert.Equal(t, []int64{1, 2, 3}, acker.getLedgerIDs()) + for i := 1; i <= 3; i++ { + assert.False(t, tracker.isDuplicate(&messageID{ledgerID: int64(i)})) + } + + tracker.add(&messageID{ledgerID: 4}) + // 4 won't be added because the cache is not full + assert.Equal(t, []int64{1, 2, 3}, acker.getLedgerIDs()) + + assert.False(t, tracker.isDuplicate(&messageID{ledgerID: 5})) + tracker.addCumulative(&messageID{ledgerID: 5}) + for i := 0; i <= 5; i++ { + assert.True(t, tracker.isDuplicate(&messageID{ledgerID: int64(i)})) + } + assert.Equal(t, int64(5), acker.getCumulativeLedgerID()) + assert.False(t, tracker.isDuplicate(&messageID{ledgerID: int64(6)})) + + tracker.flush() + assert.Eventually(t, func() bool { return len(acker.getLedgerIDs()) > 3 }, + 10*time.Millisecond, 2*time.Millisecond) + assert.Equal(t, []int64{1, 2, 3, 4}, acker.getLedgerIDs()) +} + +func TestTimedTrackerIndividualAck(t *testing.T) { + var acker mockAcker + // MaxSize: 1000, MaxTime: 100ms + tracker := newAckGroupingTracker(nil, func(id MessageID) { acker.ack(id) }, nil) + + expected := make([]int64, 0) + for i := 0; i < 999; i++ { + tracker.add(&messageID{ledgerID: int64(i)}) + expected = append(expected, int64(i)) + } + assert.Equal(t, 0, len(acker.getLedgerIDs())) + + // case 1: flush because the tracker timed out + assert.Eventually(t, func() bool { return len(acker.getLedgerIDs()) == 999 }, + 150*time.Millisecond, 10*time.Millisecond) + assert.Equal(t, expected, acker.getLedgerIDs()) + + // case 2: flush because cache is full + time.Sleep(50) // see case 3 + acker.reset() + expected = append(expected, 999) + for i := 0; i < 1001; i++ { + tracker.add(&messageID{ledgerID: int64(i)}) + } + assert.Equal(t, expected, acker.getLedgerIDs()) + + // case 3: flush will reset the timer + start := time.Now() + assert.Eventually(t, func() bool { return len(acker.getLedgerIDs()) > 1000 }, + 150*time.Millisecond, 10*time.Millisecond) + elapsed := time.Since(start) + assert.GreaterOrEqual(t, elapsed, int64(100), "elapsed", elapsed) + assert.Equal(t, append(expected, 1000), acker.getLedgerIDs()) +} + +func TestTimedTrackerCumulativeAck(t *testing.T) { + var acker mockAcker + // MaxTime is 100ms + tracker := newAckGroupingTracker(nil, nil, func(id MessageID) { acker.ackCumulative(id) }) + + // case 1: flush because of the timeout + tracker.addCumulative(&messageID{ledgerID: 1}) + assert.NotEqual(t, int64(1), acker.getCumulativeLedgerID()) + assert.Eventually(t, func() bool { return acker.getCumulativeLedgerID() == int64(1) }, + 150*time.Millisecond, 10*time.Millisecond) + assert.Equal(t, int64(1), acker.getCumulativeLedgerID()) + + // case 2: flush manually + tracker.addCumulative(&messageID{ledgerID: 2}) + tracker.flush() + assert.Equal(t, int64(2), acker.getCumulativeLedgerID()) + + // case 3: older MessageID cannot be acknowledged + tracker.addCumulative(&messageID{ledgerID: 1}) + tracker.flush() + assert.Equal(t, int64(2), acker.getCumulativeLedgerID()) +} + +func TestTimedTrackerIsDuplicate(t *testing.T) { + tracker := newAckGroupingTracker(nil, func(id MessageID) {}, func(id MessageID) {}) + + tracker.add(messageID{batchIdx: 0, batchSize: 3}) + tracker.add(messageID{batchIdx: 2, batchSize: 3}) + assert.True(t, tracker.isDuplicate(messageID{batchIdx: 0, batchSize: 3})) + assert.False(t, tracker.isDuplicate(messageID{batchIdx: 1, batchSize: 3})) + assert.True(t, tracker.isDuplicate(messageID{batchIdx: 2, batchSize: 3})) + + tracker.flush() + assert.False(t, tracker.isDuplicate(messageID{batchIdx: 0, batchSize: 3})) + assert.False(t, tracker.isDuplicate(messageID{batchIdx: 1, batchSize: 3})) + assert.False(t, tracker.isDuplicate(messageID{batchIdx: 2, batchSize: 3})) +} diff --git a/pulsar/consumer.go b/pulsar/consumer.go index 8bae57d950..9576d7aabc 100644 --- a/pulsar/consumer.go +++ b/pulsar/consumer.go @@ -80,6 +80,20 @@ type DLQPolicy struct { RetryLetterTopic string } +// AckGroupingOptions controls how to group ACK requests +// If maxSize is 0 or 1, any ACK request will be sent immediately. +// Otherwise, the ACK requests will be cached until one of the following conditions meets: +// 1. There are `MaxSize` pending ACK requests. +// 2. `MaxTime` is greater than 1 microsecond and ACK requests have been cached for `maxTime`. +// Specially, for cumulative acknowledgment, only the latest ACK is cached and it will only be sent after `MaxTime`. +type AckGroupingOptions struct { + // The maximum number of ACK requests to cache + MaxSize uint32 + + // The maximum time to cache ACK requests + MaxTime time.Duration +} + // ConsumerOptions is used to configure and create instances of Consumer. type ConsumerOptions struct { // Topic specifies the topic this consumer will subscribe on. @@ -215,6 +229,13 @@ type ConsumerOptions struct { // Enable or disable batch index acknowledgment. To enable this feature, ensure batch index acknowledgment // is enabled on the broker side. (default: false) EnableBatchIndexAcknowledgment bool + + // Controls how to group ACK requests, the default value is nil, which means: + // MaxSize: 1000 + // MaxTime: 100*time.Millisecond + // NOTE: This option does not work if AckWithResponse is true + // because there are only synchronous APIs for acknowledgment + AckGroupingOptions *AckGroupingOptions } // Consumer is an interface that abstracts behavior of Pulsar's consumer diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index bf136c8e83..8ee1822c1b 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -398,6 +398,7 @@ func (c *consumer) internalTopicSubscribeToPartitions() error { autoAckIncompleteChunk: c.options.AutoAckIncompleteChunk, consumerEventListener: c.options.EventListener, enableBatchIndexAck: c.options.EnableBatchIndexAcknowledgment, + ackGroupingOptions: c.options.AckGroupingOptions, } cons, err := newPartitionConsumer(c, c.client, opts, c.messageCh, c.dlq, c.metrics) ch <- ConsumerError{ diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index e723f8afa6..95a8d3240d 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -116,6 +116,7 @@ type partitionConsumerOpts struct { // in failover mode, this callback will be called when consumer change consumerEventListener ConsumerEventListener enableBatchIndexAck bool + ackGroupingOptions *AckGroupingOptions } type ConsumerEventListener interface { @@ -167,6 +168,7 @@ type partitionConsumer struct { chunkedMsgCtxMap *chunkedMsgCtxMap unAckChunksTracker *unAckChunksTracker + ackGroupingTracker ackGroupingTracker } func (pc *partitionConsumer) ActiveConsumerChanged(isActive bool) { @@ -310,6 +312,9 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon pc.availablePermits = &availablePermits{pc: pc} pc.chunkedMsgCtxMap = newChunkedMsgCtxMap(options.maxPendingChunkedMessage, pc) pc.unAckChunksTracker = newUnAckChunksTracker(pc) + pc.ackGroupingTracker = newAckGroupingTracker(options.ackGroupingOptions, + func(id MessageID) { pc.sendIndividualAck(id) }, + func(id MessageID) { pc.sendCumulativeAck(id) }) pc.setConsumerState(consumerInit) pc.log = client.log.SubLogger(log.Fields{ "name": pc.name, @@ -467,30 +472,35 @@ func (pc *partitionConsumer) ackID(msgID MessageID, withResponse bool) error { return errors.New("failed to convert trackingMessageID") } - ackReq := new(ackRequest) - ackReq.doneCh = make(chan struct{}) - ackReq.ackType = individualAck if !trackingID.Undefined() && trackingID.ack() { pc.metrics.AcksCounter.Inc() pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9) - ackReq.msgID = trackingID - // send ack request to eventsCh - pc.eventsCh <- ackReq - - if withResponse { - <-ackReq.doneCh - } - - pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID) - } else if pc.options.enableBatchIndexAck { - ackReq.msgID = trackingID - pc.eventsCh <- ackReq + } else if !pc.options.enableBatchIndexAck { + return nil } + var ackReq *ackRequest if withResponse { - return ackReq.err + ackReq := pc.sendIndividualAck(&trackingID) + <-ackReq.doneCh + } else { + pc.ackGroupingTracker.add(&trackingID) } - return nil + pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID) + if ackReq == nil { + return nil + } + return ackReq.err +} + +func (pc *partitionConsumer) sendIndividualAck(msgID MessageID) *ackRequest { + ackReq := &ackRequest{ + doneCh: make(chan struct{}), + ackType: individualAck, + msgID: *msgID.(*trackingMessageID), + } + pc.eventsCh <- ackReq + return ackReq } func (pc *partitionConsumer) AckIDWithResponse(msgID MessageID) error { @@ -524,14 +534,12 @@ func (pc *partitionConsumer) internalAckIDCumulative(msgID MessageID, withRespon return nil } - ackReq := new(ackRequest) - ackReq.doneCh = make(chan struct{}) - ackReq.ackType = cumulativeAck + var msgIDToAck trackingMessageID if trackingID.ackCumulative() || pc.options.enableBatchIndexAck { - ackReq.msgID = trackingID + msgIDToAck = trackingID } else if !trackingID.tracker.hasPrevBatchAcked() { // get previous batch message id - ackReq.msgID = trackingID.prev() + msgIDToAck = trackingID.prev() trackingID.tracker.setPrevBatchAcked() } else { // waiting for all the msgs are acked in this batch @@ -540,12 +548,13 @@ func (pc *partitionConsumer) internalAckIDCumulative(msgID MessageID, withRespon pc.metrics.AcksCounter.Inc() pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9) - // send ack request to eventsCh - pc.eventsCh <- ackReq + var ackReq *ackRequest if withResponse { - // wait for the request to complete if withResponse set true + ackReq := pc.sendCumulativeAck(&msgIDToAck) <-ackReq.doneCh + } else { + pc.ackGroupingTracker.addCumulative(&msgIDToAck) } pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID) @@ -554,7 +563,20 @@ func (pc *partitionConsumer) internalAckIDCumulative(msgID MessageID, withRespon pc.unAckChunksTracker.remove(cmid) } - return nil + if ackReq == nil { + return nil + } + return ackReq.err +} + +func (pc *partitionConsumer) sendCumulativeAck(msgID MessageID) *ackRequest { + ackReq := &ackRequest{ + doneCh: make(chan struct{}), + ackType: cumulativeAck, + msgID: *msgID.(*trackingMessageID), + } + pc.eventsCh <- ackReq + return ackReq } func (pc *partitionConsumer) NackID(msgID MessageID) { @@ -631,6 +653,9 @@ func (pc *partitionConsumer) Close() { return } + // flush all pending ACK requests and terminate the timer goroutine + pc.ackGroupingTracker.close() + // close chunkedMsgCtxMap pc.chunkedMsgCtxMap.Close() @@ -658,6 +683,7 @@ func (pc *partitionConsumer) Seek(msgID MessageID) error { return errors.New("unhandled messageID type") } + pc.ackGroupingTracker.flushAndClean() pc.eventsCh <- req // wait for the request to complete @@ -715,6 +741,7 @@ func (pc *partitionConsumer) SeekByTime(time time.Time) error { doneCh: make(chan struct{}), publishTime: time, } + pc.ackGroupingTracker.flushAndClean() pc.eventsCh <- req // wait for the request to complete @@ -957,6 +984,10 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header msgID = trackingMsgID } + if pc.ackGroupingTracker.isDuplicate(msgID) { + continue + } + var messageIndex *uint64 var brokerPublishTime *time.Time if brokerMetadata != nil { diff --git a/pulsar/consumer_partition_test.go b/pulsar/consumer_partition_test.go index fd50b703fb..b9a9a02c3f 100644 --- a/pulsar/consumer_partition_test.go +++ b/pulsar/consumer_partition_test.go @@ -37,6 +37,8 @@ func TestSingleMessageIDNoAckTracker(t *testing.T) { metrics: newTestMetrics(), decryptor: crypto.NewNoopDecryptor(), } + pc.ackGroupingTracker = newAckGroupingTracker(&AckGroupingOptions{MaxSize: 0}, + func(id MessageID) { pc.sendIndividualAck(id) }, nil) headersAndPayload := internal.NewBufferWrapper(rawCompatSingleMessage) if err := pc.MessageReceived(nil, headersAndPayload); err != nil { @@ -73,6 +75,8 @@ func TestBatchMessageIDNoAckTracker(t *testing.T) { metrics: newTestMetrics(), decryptor: crypto.NewNoopDecryptor(), } + pc.ackGroupingTracker = newAckGroupingTracker(&AckGroupingOptions{MaxSize: 0}, + func(id MessageID) { pc.sendIndividualAck(id) }, nil) headersAndPayload := internal.NewBufferWrapper(rawBatchMessage1) if err := pc.MessageReceived(nil, headersAndPayload); err != nil { @@ -105,6 +109,8 @@ func TestBatchMessageIDWithAckTracker(t *testing.T) { metrics: newTestMetrics(), decryptor: crypto.NewNoopDecryptor(), } + pc.ackGroupingTracker = newAckGroupingTracker(&AckGroupingOptions{MaxSize: 0}, + func(id MessageID) { pc.sendIndividualAck(id) }, nil) headersAndPayload := internal.NewBufferWrapper(rawBatchMessage10) if err := pc.MessageReceived(nil, headersAndPayload); err != nil { diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 0eb7aae572..de90c0e7dd 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -3853,37 +3853,38 @@ func TestAckWithMessageID(t *testing.T) { } func TestBatchIndexAck(t *testing.T) { - tests := []struct { - AckWithResponse bool - Cumulative bool - }{ - { - AckWithResponse: true, - Cumulative: true, - }, - { - AckWithResponse: true, - Cumulative: false, - }, - { - AckWithResponse: false, - Cumulative: true, - }, - { - AckWithResponse: false, - Cumulative: false, - }, - } - for _, params := range tests { - t.Run(fmt.Sprintf("TestBatchIndexAck_WithResponse_%v_Cumulative_%v", - params.AckWithResponse, params.Cumulative), + type config struct { + ackWithResponse bool + cumulative bool + ackGroupingOptions *AckGroupingOptions + } + configs := make([]config, 0) + for _, option := range []*AckGroupingOptions{ + nil, // MaxSize: 1000, MaxTime: 10ms + {MaxSize: 0, MaxTime: 0}, + {MaxSize: 1000, MaxTime: 0}, + } { + configs = append(configs, config{true, true, option}) + configs = append(configs, config{true, false, option}) + configs = append(configs, config{false, true, option}) + configs = append(configs, config{false, false, option}) + } + + for _, params := range configs { + option := params.ackGroupingOptions + if option == nil { + option = &AckGroupingOptions{1000, 10 * time.Millisecond} + } + + t.Run(fmt.Sprintf("TestBatchIndexAck_WithResponse_%v_Cumulative_%v_AckGroupingOption_%v_%v", + params.ackWithResponse, params.cumulative, option.MaxSize, option.MaxTime.Milliseconds()), func(t *testing.T) { - runBatchIndexAckTest(t, params.AckWithResponse, params.Cumulative) + runBatchIndexAckTest(t, params.ackWithResponse, params.cumulative, params.ackGroupingOptions) }) } } -func runBatchIndexAckTest(t *testing.T, ackWithResponse bool, cumulative bool) { +func runBatchIndexAckTest(t *testing.T, ackWithResponse bool, cumulative bool, option *AckGroupingOptions) { client, err := NewClient(ClientOptions{ URL: lookupURL, }) @@ -3897,6 +3898,7 @@ func runBatchIndexAckTest(t *testing.T, ackWithResponse bool, cumulative bool) { SubscriptionName: "my-sub", AckWithResponse: ackWithResponse, EnableBatchIndexAcknowledgment: true, + AckGroupingOptions: option, }) assert.Nil(t, err) return consumer diff --git a/pulsar/message.go b/pulsar/message.go index d37692b1a4..c44957d18d 100644 --- a/pulsar/message.go +++ b/pulsar/message.go @@ -181,3 +181,42 @@ func EarliestMessageID() MessageID { func LatestMessageID() MessageID { return latestMessageID } + +func messageIDCompare(lhs MessageID, rhs MessageID) int { + if lhs.LedgerID() < rhs.LedgerID() { + return -1 + } else if lhs.LedgerID() > rhs.LedgerID() { + return 1 + } + if lhs.EntryID() < rhs.EntryID() { + return -1 + } else if lhs.EntryID() > rhs.EntryID() { + return 1 + } + // When performing batch index ACK on a batched message whose batch size is N, + // the ACK order should be: + // (ledger, entry, 0) -> (ledger, entry, 1) -> ... -> (ledger, entry, N-1) -> (ledger, entry) + // So we have to treat any MessageID with the batch index precedes the MessageID without the batch index + // if they are in the same entry. + if lhs.BatchIdx() < 0 && rhs.BatchIdx() < 0 { + return 0 + } else if lhs.BatchIdx() >= 0 && rhs.BatchIdx() < 0 { + return -1 + } else if lhs.BatchIdx() < 0 && rhs.BatchIdx() >= 0 { + return 1 + } + if lhs.BatchIdx() < rhs.BatchIdx() { + return -1 + } else if lhs.BatchIdx() > rhs.BatchIdx() { + return 1 + } + return 0 +} + +func messageIDHash(id MessageID) int64 { + return id.LedgerID() + 31*id.EntryID() +} + +func messageIDIsBatch(id MessageID) bool { + return id.BatchIdx() >= 0 && id.BatchSize() > 0 +} From e2ea255052e8a527091791ef368851d885ee2d45 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 27 Feb 2023 10:38:49 +0800 Subject: [PATCH 018/348] [feat][txn]Implement transactionCoordinatorClient (#953) * [feat][txn]Implement transactionCoordinatorClient Master Issue:/~https://github.com/apache/pulsar-client-go/issues/932 ### Motivation Implement transaction coordinator client. ### Modifications 1. Implement transaction coordinator 2. implement transaction API 3. Add metric and test * Fix checkstyle and CI * fix some comments * fix some comments * fix some comments * comment style consistent * comment style consistent * fix some comments --- Makefile | 2 +- integration-tests/conf/standalone.conf | 27 ++- pulsar/client.go | 2 + pulsar/client_impl.go | 9 + pulsar/client_impl_test.go | 37 ++++- pulsar/error.go | 7 + pulsar/helper_for_test.go | 7 + pulsar/internal/commands.go | 11 ++ pulsar/internal/connection.go | 11 +- pulsar/transaction.go | 23 +++ pulsar/transaction_coordinator_client.go | 199 +++++++++++++++++++++++ pulsar/transaction_test.go | 98 +++++++++++ scripts/pulsar-test-service-start.sh | 14 +- 13 files changed, 422 insertions(+), 25 deletions(-) create mode 100644 pulsar/transaction.go create mode 100644 pulsar/transaction_coordinator_client.go create mode 100644 pulsar/transaction_test.go diff --git a/Makefile b/Makefile index 3245434736..231c05eb55 100644 --- a/Makefile +++ b/Makefile @@ -18,7 +18,7 @@ # IMAGE_NAME = pulsar-client-go-test:latest -PULSAR_VERSION ?= 2.8.3 +PULSAR_VERSION ?= 2.10.3 PULSAR_IMAGE = apachepulsar/pulsar:$(PULSAR_VERSION) GO_VERSION ?= 1.18 GOLANG_IMAGE = golang:$(GO_VERSION) diff --git a/integration-tests/conf/standalone.conf b/integration-tests/conf/standalone.conf index b9ff87bedb..334fc87d2a 100644 --- a/integration-tests/conf/standalone.conf +++ b/integration-tests/conf/standalone.conf @@ -96,8 +96,7 @@ anonymousUserRole=anonymous # Enable authentication authenticationEnabled=true - -# Autentication provider name list, which is comma separated list of class names +# Authentication provider name list, which is comma separated list of class names authenticationProviders=org.apache.pulsar.broker.authentication.AuthenticationProviderTls,org.apache.pulsar.broker.authentication.AuthenticationProviderToken,org.apache.pulsar.broker.authentication.AuthenticationProviderBasic # Enforce authorization @@ -111,8 +110,10 @@ superUserRoles=localhost,superUser,admin # Authentication settings of the broker itself. Used when the broker connects to other brokers, # either in same or other clusters -brokerClientAuthenticationPlugin= -brokerClientAuthenticationParameters= +brokerClientTlsEnabled=true +brokerClientTrustCertsFilePath=/pulsar/certs/cacert.pem +brokerClientAuthenticationPlugin=org.apache.pulsar.client.impl.auth.AuthenticationTls +brokerClientAuthenticationParameters={"tlsCertFile":"/pulsar/certs/client-cert.pem","tlsKeyFile":"/pulsar/certs/client-key.pem"} ### --- BookKeeper Client --- ### @@ -294,3 +295,21 @@ globalZookeeperServers= brokerServicePurgeInactiveFrequencyInSeconds=60 acknowledgmentAtBatchIndexLevelEnabled=true +### --- Transaction config variables --- ### +# Enable transaction coordinator in broker +transactionCoordinatorEnabled=true +transactionMetadataStoreProviderClassName=org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider + +# Transaction buffer take snapshot transaction count +transactionBufferSnapshotMaxTransactionCount=1000 + +# Transaction buffer take snapshot interval time +# Unit : millisecond +transactionBufferSnapshotMinTimeInMillis=5000 + +# Enable or disable system topic +systemTopicEnabled=true + +# The schema compatibility strategy is used for system topics. +# Available values: ALWAYS_INCOMPATIBLE, ALWAYS_COMPATIBLE, BACKWARD, FORWARD, FULL, BACKWARD_TRANSITIVE, FORWARD_TRANSITIVE, FULL_TRANSITIVE +systemTopicSchemaCompatibilityStrategy=ALWAYS_COMPATIBLE \ No newline at end of file diff --git a/pulsar/client.go b/pulsar/client.go index d75c977b40..135d22b473 100644 --- a/pulsar/client.go +++ b/pulsar/client.go @@ -142,6 +142,8 @@ type ClientOptions struct { // Specify metric registerer used to register metrics. // Default prometheus.DefaultRegisterer MetricsRegisterer prometheus.Registerer + + EnableTransaction bool } // Client represents a pulsar client diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index e794e3d71e..f4448047ee 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -41,6 +41,7 @@ type client struct { handlers internal.ClientHandlers lookupService internal.LookupService metrics *internal.Metrics + tcClient *transactionCoordinatorClient log log.Logger } @@ -162,6 +163,14 @@ func newClient(options ClientOptions) (Client, error) { c.handlers = internal.NewClientHandlers() + if options.EnableTransaction { + c.tcClient = newTransactionCoordinatorClientImpl(c) + err = c.tcClient.start() + if err != nil { + return nil, err + } + } + return c, nil } diff --git a/pulsar/client_impl_test.go b/pulsar/client_impl_test.go index 299203f091..b8efe9a631 100644 --- a/pulsar/client_impl_test.go +++ b/pulsar/client_impl_test.go @@ -24,6 +24,7 @@ import ( "net/http" "net/http/httptest" "os" + "strings" "testing" "time" @@ -425,7 +426,7 @@ func TestNamespaceTopics(t *testing.T) { t.Fatal(err) } topic2 := fmt.Sprintf("%s/topic-2", namespace) - if err := httpPut("admin/v2/persistent/"+topic2, namespace); err != nil { + if err := httpPut("admin/v2/persistent/"+topic2, nil); err != nil { t.Fatal(err) } defer func() { @@ -446,7 +447,13 @@ func TestNamespaceTopics(t *testing.T) { if err != nil { t.Fatal(err) } - assert.Equal(t, 2, len(topics)) + topicCount := 0 + for _, value := range topics { + if !strings.Contains(value, "__transaction_buffer_snapshot") { + topicCount++ + } + } + assert.Equal(t, 2, topicCount) // add a non-persistent topic topicName := fmt.Sprintf("non-persistent://%s/testNonPersistentTopic", namespace) @@ -467,7 +474,13 @@ func TestNamespaceTopics(t *testing.T) { if err != nil { t.Fatal(err) } - assert.Equal(t, 2, len(topics)) + topicCount = 0 + for _, value := range topics { + if !strings.Contains(value, "__transaction_buffer_snapshot") { + topicCount++ + } + } + assert.Equal(t, 2, topicCount) } func TestNamespaceTopicsWebURL(t *testing.T) { @@ -488,7 +501,7 @@ func TestNamespaceTopicsWebURL(t *testing.T) { t.Fatal(err) } topic2 := fmt.Sprintf("%s/topic-2", namespace) - if err := httpPut("admin/v2/persistent/"+topic2, namespace); err != nil { + if err := httpPut("admin/v2/persistent/"+topic2, nil); err != nil { t.Fatal(err) } defer func() { @@ -509,7 +522,13 @@ func TestNamespaceTopicsWebURL(t *testing.T) { if err != nil { t.Fatal(err) } - assert.Equal(t, 2, len(topics)) + topicCount := 0 + for _, value := range topics { + if !strings.Contains(value, "__transaction_buffer_snapshot") { + topicCount++ + } + } + assert.Equal(t, 2, topicCount) // add a non-persistent topic topicName := fmt.Sprintf("non-persistent://%s/testNonPersistentTopic", namespace) @@ -530,7 +549,13 @@ func TestNamespaceTopicsWebURL(t *testing.T) { if err != nil { t.Fatal(err) } - assert.Equal(t, 2, len(topics)) + topicCount = 0 + for _, value := range topics { + if !strings.Contains(value, "__transaction_buffer_snapshot") { + topicCount++ + } + } + assert.Equal(t, 2, topicCount) } func anonymousNamespacePolicy() map[string]interface{} { diff --git a/pulsar/error.go b/pulsar/error.go index ead5cf9436..ce366f5edc 100644 --- a/pulsar/error.go +++ b/pulsar/error.go @@ -103,6 +103,13 @@ const ( ProducerClosed // SchemaFailure means the payload could not be encoded using the Schema SchemaFailure + + // ReachMaxPendingOps means the pending operations in transaction_impl coordinator reach the maximum. + ReachMaxPendingOps + // InvalidStatus means the component status is not as expected. + InvalidStatus + // TransactionError means this is a transaction related error + TransactionError ) // Error implement error interface, composed of two parts: msg and result. diff --git a/pulsar/helper_for_test.go b/pulsar/helper_for_test.go index a2888d0d58..eda2d72b4e 100644 --- a/pulsar/helper_for_test.go +++ b/pulsar/helper_for_test.go @@ -157,6 +157,13 @@ func topicStats(topic string) (map[string]interface{}, error) { return metadata, err } +func transactionStats(id *TxnID) (map[string]interface{}, error) { + var metadata map[string]interface{} + path := fmt.Sprintf("admin/v3/transactions/transactionMetadata/%d/%d", id.mostSigBits, id.leastSigBits) + err := httpGet(path, &metadata) + return metadata, err +} + func topicPath(topic string) string { tn, _ := internal.ParseTopicName(topic) idx := strings.LastIndex(tn.Name, "/") diff --git a/pulsar/internal/commands.go b/pulsar/internal/commands.go index 70acf39e07..00e075beee 100644 --- a/pulsar/internal/commands.go +++ b/pulsar/internal/commands.go @@ -218,6 +218,17 @@ func baseCommand(cmdType pb.BaseCommand_Type, msg proto.Message) *pb.BaseCommand cmd.GetOrCreateSchema = msg.(*pb.CommandGetOrCreateSchema) case pb.BaseCommand_GET_SCHEMA: cmd.GetSchema = msg.(*pb.CommandGetSchema) + case pb.BaseCommand_TC_CLIENT_CONNECT_REQUEST: + cmd.TcClientConnectRequest = msg.(*pb.CommandTcClientConnectRequest) + case pb.BaseCommand_NEW_TXN: + cmd.NewTxn = msg.(*pb.CommandNewTxn) + case pb.BaseCommand_ADD_PARTITION_TO_TXN: + cmd.AddPartitionToTxn = msg.(*pb.CommandAddPartitionToTxn) + case pb.BaseCommand_ADD_SUBSCRIPTION_TO_TXN: + cmd.AddSubscriptionToTxn = msg.(*pb.CommandAddSubscriptionToTxn) + case pb.BaseCommand_END_TXN: + cmd.EndTxn = msg.(*pb.CommandEndTxn) + default: panic(fmt.Sprintf("Missing command type: %v", cmdType)) } diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 67b6f32b43..09623e50d8 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -583,7 +583,16 @@ func (c *connection) internalReceivedCommand(cmd *pb.BaseCommand, headersAndPayl c.handlePing() case pb.BaseCommand_PONG: c.handlePong() - + case pb.BaseCommand_TC_CLIENT_CONNECT_RESPONSE: + c.handleResponse(cmd.TcClientConnectResponse.GetRequestId(), cmd) + case pb.BaseCommand_NEW_TXN_RESPONSE: + c.handleResponse(cmd.NewTxnResponse.GetRequestId(), cmd) + case pb.BaseCommand_ADD_PARTITION_TO_TXN_RESPONSE: + c.handleResponse(cmd.AddPartitionToTxnResponse.GetRequestId(), cmd) + case pb.BaseCommand_ADD_SUBSCRIPTION_TO_TXN_RESPONSE: + c.handleResponse(cmd.AddSubscriptionToTxnResponse.GetRequestId(), cmd) + case pb.BaseCommand_END_TXN_RESPONSE: + c.handleResponse(cmd.EndTxnResponse.GetRequestId(), cmd) case pb.BaseCommand_ACTIVE_CONSUMER_CHANGE: c.handleActiveConsumerChange(cmd.GetActiveConsumerChange()) diff --git a/pulsar/transaction.go b/pulsar/transaction.go new file mode 100644 index 0000000000..ae7c673046 --- /dev/null +++ b/pulsar/transaction.go @@ -0,0 +1,23 @@ +// 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 pulsar + +type TxnID struct { + mostSigBits uint64 + leastSigBits uint64 +} diff --git a/pulsar/transaction_coordinator_client.go b/pulsar/transaction_coordinator_client.go new file mode 100644 index 0000000000..82d1490398 --- /dev/null +++ b/pulsar/transaction_coordinator_client.go @@ -0,0 +1,199 @@ +// 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 pulsar + +import ( + "context" + "strconv" + "sync/atomic" + "time" + + "github.com/apache/pulsar-client-go/pulsar/internal" + pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" + "github.com/apache/pulsar-client-go/pulsar/log" + "google.golang.org/protobuf/proto" +) + +type transactionCoordinatorClient struct { + client *client + cons []internal.Connection + epoch uint64 + semaphore internal.Semaphore + log log.Logger +} + +// TransactionCoordinatorAssign is the transaction_impl coordinator topic which is used to look up the broker +// where the TC located. +const TransactionCoordinatorAssign = "persistent://pulsar/system/transaction_coordinator_assign" + +// newTransactionCoordinatorClientImpl init a transactionImpl coordinator client and +// acquire connections with all transactionImpl coordinators. +func newTransactionCoordinatorClientImpl(client *client) *transactionCoordinatorClient { + tc := &transactionCoordinatorClient{ + client: client, + semaphore: internal.NewSemaphore(1000), + } + tc.log = client.log.SubLogger(log.Fields{}) + return tc +} + +func (tc *transactionCoordinatorClient) start() error { + r, err := tc.client.lookupService.GetPartitionedTopicMetadata(TransactionCoordinatorAssign) + if err != nil { + return err + } + tc.cons = make([]internal.Connection, r.Partitions) + + //Get connections with all transaction_impl coordinators which is synchronized + for i := 0; i < r.Partitions; i++ { + err := tc.grabConn(uint64(i)) + if err != nil { + return err + } + } + return nil +} + +func (tc *transactionCoordinatorClient) grabConn(partition uint64) error { + lr, err := tc.client.lookupService.Lookup(getTCAssignTopicName(partition)) + if err != nil { + tc.log.WithError(err).Warn("Failed to lookup the transaction_impl " + + "coordinator assign topic [" + strconv.FormatUint(partition, 10) + "]") + return err + } + + requestID := tc.client.rpcClient.NewRequestID() + cmdTCConnect := pb.CommandTcClientConnectRequest{ + RequestId: proto.Uint64(requestID), + TcId: proto.Uint64(partition), + } + + res, err := tc.client.rpcClient.Request(lr.LogicalAddr, lr.PhysicalAddr, requestID, + pb.BaseCommand_TC_CLIENT_CONNECT_REQUEST, &cmdTCConnect) + + if err != nil { + tc.log.WithError(err).Error("Failed to connect transaction_impl coordinator " + + strconv.FormatUint(partition, 10)) + return err + } + tc.cons[partition] = res.Cnx + return nil +} + +func (tc *transactionCoordinatorClient) close() { + for _, con := range tc.cons { + con.Close() + } +} + +// newTransaction new a transactionImpl which can be used to guarantee exactly-once semantics. +func (tc *transactionCoordinatorClient) newTransaction(timeout time.Duration) (*TxnID, error) { + if err := tc.canSendRequest(); err != nil { + return nil, err + } + requestID := tc.client.rpcClient.NewRequestID() + nextTcID := tc.nextTCNumber() + cmdNewTxn := &pb.CommandNewTxn{ + RequestId: proto.Uint64(requestID), + TcId: proto.Uint64(nextTcID), + TxnTtlSeconds: proto.Uint64(uint64(timeout.Milliseconds())), + } + + cnx, err := tc.client.rpcClient.RequestOnCnx(tc.cons[nextTcID], requestID, pb.BaseCommand_NEW_TXN, cmdNewTxn) + tc.semaphore.Release() + if err != nil { + return nil, err + } + + return &TxnID{*cnx.Response.NewTxnResponse.TxnidMostBits, + *cnx.Response.NewTxnResponse.TxnidLeastBits}, nil +} + +// addPublishPartitionToTxn register the partitions which published messages with the transactionImpl. +// And this can be used when ending the transactionImpl. +func (tc *transactionCoordinatorClient) addPublishPartitionToTxn(id *TxnID, partitions []string) error { + if err := tc.canSendRequest(); err != nil { + return err + } + requestID := tc.client.rpcClient.NewRequestID() + cmdAddPartitions := &pb.CommandAddPartitionToTxn{ + RequestId: proto.Uint64(requestID), + TxnidMostBits: proto.Uint64(id.mostSigBits), + TxnidLeastBits: proto.Uint64(id.leastSigBits), + Partitions: partitions, + } + _, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, + pb.BaseCommand_ADD_PARTITION_TO_TXN, cmdAddPartitions) + tc.semaphore.Release() + return err +} + +// addSubscriptionToTxn register the subscription which acked messages with the transactionImpl. +// And this can be used when ending the transactionImpl. +func (tc *transactionCoordinatorClient) addSubscriptionToTxn(id *TxnID, topic string, subscription string) error { + if err := tc.canSendRequest(); err != nil { + return err + } + requestID := tc.client.rpcClient.NewRequestID() + sub := &pb.Subscription{ + Topic: &topic, + Subscription: &subscription, + } + cmdAddSubscription := &pb.CommandAddSubscriptionToTxn{ + RequestId: proto.Uint64(requestID), + TxnidMostBits: proto.Uint64(id.mostSigBits), + TxnidLeastBits: proto.Uint64(id.leastSigBits), + Subscription: []*pb.Subscription{sub}, + } + _, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, + pb.BaseCommand_ADD_SUBSCRIPTION_TO_TXN, cmdAddSubscription) + tc.semaphore.Release() + return err +} + +// endTxn commit or abort the transactionImpl. +func (tc *transactionCoordinatorClient) endTxn(id *TxnID, action pb.TxnAction) error { + if err := tc.canSendRequest(); err != nil { + return err + } + requestID := tc.client.rpcClient.NewRequestID() + cmdEndTxn := &pb.CommandEndTxn{ + RequestId: proto.Uint64(requestID), + TxnAction: &action, + TxnidMostBits: proto.Uint64(id.mostSigBits), + TxnidLeastBits: proto.Uint64(id.leastSigBits), + } + _, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, pb.BaseCommand_END_TXN, cmdEndTxn) + tc.semaphore.Release() + return err +} + +func getTCAssignTopicName(partition uint64) string { + return TransactionCoordinatorAssign + "-partition-" + strconv.FormatUint(partition, 10) +} + +func (tc *transactionCoordinatorClient) canSendRequest() error { + if !tc.semaphore.Acquire(context.Background()) { + return newError(UnknownError, "Failed to acquire semaphore") + } + return nil +} + +func (tc *transactionCoordinatorClient) nextTCNumber() uint64 { + return atomic.AddUint64(&tc.epoch, 1) % uint64(len(tc.cons)) +} diff --git a/pulsar/transaction_test.go b/pulsar/transaction_test.go new file mode 100644 index 0000000000..14a652fdee --- /dev/null +++ b/pulsar/transaction_test.go @@ -0,0 +1,98 @@ +// 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 pulsar + +import ( + pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" + "github.com/stretchr/testify/assert" + + "testing" + "time" +) + +func TestTCClient(t *testing.T) { + //1. Prepare: create PulsarClient and init transaction coordinator client. + topic := newTopicName() + sub := "my-sub" + tc, client := createTcClient(t) + //2. Prepare: create Topic and Subscription. + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: sub, + }) + assert.NoError(t, err) + //3. Test newTransaction, addSubscriptionToTxn, addPublishPartitionToTxn + //Create a transaction1 and add subscription and publish topic to the transaction. + id1, err := tc.newTransaction(3 * time.Minute) + assert.NoError(t, err) + err = tc.addSubscriptionToTxn(id1, topic, sub) + assert.NoError(t, err) + err = tc.addPublishPartitionToTxn(id1, []string{topic}) + assert.NoError(t, err) + //4. Verify the transaction1 stats + stats, err := transactionStats(id1) + assert.NoError(t, err) + assert.Equal(t, "OPEN", stats["status"]) + producedPartitions := stats["producedPartitions"].(map[string]interface{}) + ackedPartitions := stats["ackedPartitions"].(map[string]interface{}) + _, ok := producedPartitions[topic] + assert.True(t, ok) + temp, ok := ackedPartitions[topic] + assert.True(t, ok) + subscriptions := temp.(map[string]interface{}) + _, ok = subscriptions[sub] + assert.True(t, ok) + //5. Test End transaction + //Create transaction2 and Commit the transaction. + id2, err := tc.newTransaction(3 * time.Minute) + assert.NoError(t, err) + //6. Verify the transaction2 stats + stats2, err := transactionStats(id2) + assert.NoError(t, err) + assert.Equal(t, "OPEN", stats2["status"]) + err = tc.endTxn(id2, pb.TxnAction_COMMIT) + assert.NoError(t, err) + stats2, err = transactionStats(id2) + //The transaction will be removed from txnMeta. Therefore, it is expected that stats2 is zero + if err == nil { + assert.Equal(t, "COMMITTED", stats2["status"]) + } else { + assert.Equal(t, err.Error(), "http error status code: 404") + } + defer consumer.Close() + defer tc.close() + defer client.Close() +} + +// createTcClient Create a transaction coordinator client to send request +func createTcClient(t *testing.T) (*transactionCoordinatorClient, *client) { + c, err := NewClient(ClientOptions{ + URL: webServiceURLTLS, + TLSTrustCertsFilePath: caCertsPath, + Authentication: NewAuthenticationTLS(tlsClientCertPath, tlsClientKeyPath), + }) + if err != nil { + t.Fatalf("Failed to create client due to %s", err.Error()) + } + tcClient := newTransactionCoordinatorClientImpl(c.(*client)) + if err = tcClient.start(); err != nil { + t.Fatalf("Failed to start transaction coordinator due to %s", err.Error()) + } + + return tcClient, c.(*client) +} diff --git a/scripts/pulsar-test-service-start.sh b/scripts/pulsar-test-service-start.sh index a7c4797615..ed90b0660a 100755 --- a/scripts/pulsar-test-service-start.sh +++ b/scripts/pulsar-test-service-start.sh @@ -51,19 +51,7 @@ until curl http://localhost:8080/metrics > /dev/null 2>&1 ; do sleep 1; done echo "-- Pulsar service is ready -- Configure permissions" -# Create "standalone" cluster -$PULSAR_ADMIN clusters create \ - standalone \ - --url http://localhost:8080/ \ - --url-secure https://localhost:8443/ \ - --broker-url pulsar://localhost:6650/ \ - --broker-url-secure pulsar+ssl://localhost:6651/ - -# Create "public" tenant -$PULSAR_ADMIN tenants create public -r "anonymous" -c "standalone" - -# Create "public/default" with no auth required -$PULSAR_ADMIN namespaces create public/default +$PULSAR_ADMIN tenants update public -r anonymous $PULSAR_ADMIN namespaces grant-permission public/default \ --actions produce,consume \ --role "anonymous" From d60bff69d2e59228cad64be74a2d31256fd3a5fa Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 1 Mar 2023 09:19:16 +0800 Subject: [PATCH 019/348] Fix flaky Key_Shared subscription related tests (#970) ### Motivation After /~https://github.com/apache/pulsar-client-go/pull/953, the Pulsar version was upgraded from 2.8.3 to 2.10.3. However, [PIP-119](/~https://github.com/apache/pulsar/pull/13352) changed the default value of `subscriptionKeySharedUseConsistentHashing` to true, which leads to the flakiness of Key_Shared subscription related tests. Example: /~https://github.com/apache/pulsar-client-go/actions/runs/4291098473/jobs/7475868787 ### Modifications Configure `subscriptionKeySharedUseConsistentHashing` with `false`. --- integration-tests/conf/standalone.conf | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/integration-tests/conf/standalone.conf b/integration-tests/conf/standalone.conf index 334fc87d2a..c816c8fd16 100644 --- a/integration-tests/conf/standalone.conf +++ b/integration-tests/conf/standalone.conf @@ -312,4 +312,6 @@ systemTopicEnabled=true # The schema compatibility strategy is used for system topics. # Available values: ALWAYS_INCOMPATIBLE, ALWAYS_COMPATIBLE, BACKWARD, FORWARD, FULL, BACKWARD_TRANSITIVE, FORWARD_TRANSITIVE, FULL_TRANSITIVE -systemTopicSchemaCompatibilityStrategy=ALWAYS_COMPATIBLE \ No newline at end of file +systemTopicSchemaCompatibilityStrategy=ALWAYS_COMPATIBLE + +subscriptionKeySharedUseConsistentHashing=false From 13843e99201141257904a5157600079752665774 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 1 Mar 2023 10:52:43 +0800 Subject: [PATCH 020/348] [perf] Support batch index ACK and max number of messages in batch (#967) ### Motivation Currently, the perf tool does not support the maximum number of messages in batch for producer and batch index ACK for consumer. ### Modifications - Add the `--batching-max-size` option to configure the maximum number of messages for producer - To keep the code style consistent, change the `--batching-max-size` config from `int` to `uint` - Add the `--enable-batch-index-ack` option to enable batch index ACK for consumer. --- perf/perf-consumer.go | 13 ++++++++----- perf/perf-producer.go | 20 ++++++++++++-------- 2 files changed, 20 insertions(+), 13 deletions(-) diff --git a/perf/perf-consumer.go b/perf/perf-consumer.go index 7fb8aab8e9..6b6e411dbc 100644 --- a/perf/perf-consumer.go +++ b/perf/perf-consumer.go @@ -31,9 +31,10 @@ import ( // ConsumeArgs define the parameters required by consume type ConsumeArgs struct { - Topic string - SubscriptionName string - ReceiverQueueSize int + Topic string + SubscriptionName string + ReceiverQueueSize int + EnableBatchIndexAck bool } func newConsumerCommand() *cobra.Command { @@ -55,6 +56,7 @@ func newConsumerCommand() *cobra.Command { flags := cmd.Flags() flags.StringVarP(&consumeArgs.SubscriptionName, "subscription", "s", "sub", "Subscription name") flags.IntVarP(&consumeArgs.ReceiverQueueSize, "receiver-queue-size", "r", 1000, "Receiver queue size") + flags.BoolVar(&consumeArgs.EnableBatchIndexAck, "enable-batch-index-ack", false, "Whether to enable batch index ACK") return cmd } @@ -74,8 +76,9 @@ func consume(consumeArgs *ConsumeArgs, stop <-chan struct{}) { defer client.Close() consumer, err := client.Subscribe(pulsar.ConsumerOptions{ - Topic: consumeArgs.Topic, - SubscriptionName: consumeArgs.SubscriptionName, + Topic: consumeArgs.Topic, + SubscriptionName: consumeArgs.SubscriptionName, + EnableBatchIndexAcknowledgment: consumeArgs.EnableBatchIndexAck, }) if err != nil { diff --git a/perf/perf-producer.go b/perf/perf-producer.go index 0ee0083e4a..9d062ae105 100644 --- a/perf/perf-producer.go +++ b/perf/perf-producer.go @@ -32,12 +32,13 @@ import ( // ProduceArgs define the parameters required by produce type ProduceArgs struct { - Topic string - Rate int - BatchingTimeMillis int - BatchingMaxSize int - MessageSize int - ProducerQueueSize int + Topic string + Rate int + BatchingTimeMillis int + BatchingMaxSize uint + BatchingNumMessages uint + MessageSize int + ProducerQueueSize int } func newProducerCommand() *cobra.Command { @@ -62,8 +63,10 @@ func newProducerCommand() *cobra.Command { "Publish rate. Set to 0 to go unthrottled") flags.IntVarP(&produceArgs.BatchingTimeMillis, "batching-time", "b", 1, "Batching grouping time in millis") - flags.IntVarP(&produceArgs.BatchingMaxSize, "batching-max-size", "", 128, + flags.UintVar(&produceArgs.BatchingMaxSize, "batching-max-size", 128, "Max size of a batch (in KB)") + flags.UintVar(&produceArgs.BatchingNumMessages, "batching-num-messages", 1000, + "Maximum number of messages permitted in a batch") flags.IntVarP(&produceArgs.MessageSize, "size", "s", 1024, "Message size") flags.IntVarP(&produceArgs.ProducerQueueSize, "queue-size", "q", 1000, @@ -88,7 +91,8 @@ func produce(produceArgs *ProduceArgs, stop <-chan struct{}) { Topic: produceArgs.Topic, MaxPendingMessages: produceArgs.ProducerQueueSize, BatchingMaxPublishDelay: time.Millisecond * time.Duration(produceArgs.BatchingTimeMillis), - BatchingMaxSize: uint(produceArgs.BatchingMaxSize * 1024), + BatchingMaxSize: produceArgs.BatchingMaxSize * 1024, + BatchingMaxMessages: produceArgs.BatchingNumMessages, }) if err != nil { log.Fatal(err) From 5fa431d06dae5615fbadda78e9c8ef68de0aab4f Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Wed, 1 Mar 2023 16:33:22 +0800 Subject: [PATCH 021/348] [feat] Support memory limit for producer. (#955) * [feat] Support memory limit for producer. * Fix code reviews. * Change channel_cond access level * Change name and add note. * change name. --- pulsar/client.go | 4 + pulsar/client_impl.go | 12 +- pulsar/error.go | 5 + pulsar/internal/channel_cond.go | 76 ++++++++ pulsar/internal/channel_cond_test.go | 55 ++++++ pulsar/internal/memory_limit_controller.go | 101 ++++++++++ .../internal/memory_limit_controller_test.go | 184 ++++++++++++++++++ pulsar/producer_partition.go | 68 ++++--- pulsar/producer_test.go | 126 ++++++++++++ 9 files changed, 606 insertions(+), 25 deletions(-) create mode 100644 pulsar/internal/channel_cond.go create mode 100644 pulsar/internal/channel_cond_test.go create mode 100644 pulsar/internal/memory_limit_controller.go create mode 100644 pulsar/internal/memory_limit_controller_test.go diff --git a/pulsar/client.go b/pulsar/client.go index 135d22b473..75b363d7cc 100644 --- a/pulsar/client.go +++ b/pulsar/client.go @@ -144,6 +144,10 @@ type ClientOptions struct { MetricsRegisterer prometheus.Registerer EnableTransaction bool + + // Limit of client memory usage (in byte). The 64M default can guarantee a high producer throughput. + // Config less than 0 indicates off memory limit. + MemoryLimitBytes int64 } // Client represents a pulsar client diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index f4448047ee..7d90922bf9 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -33,6 +33,7 @@ const ( defaultConnectionTimeout = 10 * time.Second defaultOperationTimeout = 30 * time.Second defaultKeepAliveInterval = 30 * time.Second + defaultMemoryLimitBytes = 64 * 1024 * 1024 ) type client struct { @@ -42,6 +43,7 @@ type client struct { lookupService internal.LookupService metrics *internal.Metrics tcClient *transactionCoordinatorClient + memLimit internal.MemoryLimitController log log.Logger } @@ -134,11 +136,17 @@ func newClient(options ClientOptions) (Client, error) { keepAliveInterval = defaultKeepAliveInterval } + memLimitBytes := options.MemoryLimitBytes + if memLimitBytes == 0 { + memLimitBytes = defaultMemoryLimitBytes + } + c := &client{ cnxPool: internal.NewConnectionPool(tlsConfig, authProvider, connectionTimeout, keepAliveInterval, maxConnectionsPerHost, logger, metrics), - log: logger, - metrics: metrics, + log: logger, + metrics: metrics, + memLimit: internal.NewMemoryLimitController(memLimitBytes), } serviceNameResolver := internal.NewPulsarServiceNameResolver(url) diff --git a/pulsar/error.go b/pulsar/error.go index ce366f5edc..0aa1e3c20d 100644 --- a/pulsar/error.go +++ b/pulsar/error.go @@ -110,6 +110,9 @@ const ( InvalidStatus // TransactionError means this is a transaction related error TransactionError + + // ClientMemoryBufferIsFull client limit buffer is full + ClientMemoryBufferIsFull ) // Error implement error interface, composed of two parts: msg and result. @@ -216,6 +219,8 @@ func getResultStr(r Result) string { return "ProducerClosed" case SchemaFailure: return "SchemaFailure" + case ClientMemoryBufferIsFull: + return "ClientMemoryBufferIsFull" default: return fmt.Sprintf("Result(%d)", r) } diff --git a/pulsar/internal/channel_cond.go b/pulsar/internal/channel_cond.go new file mode 100644 index 0000000000..38301abef8 --- /dev/null +++ b/pulsar/internal/channel_cond.go @@ -0,0 +1,76 @@ +// 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 ( + "context" + "sync" + "sync/atomic" + "unsafe" +) + +type chCond struct { + L sync.Locker + // The pointer to the channel, the channel pointed to may change, + // because we will use the channel's close mechanism to implement broadcast notifications. + notifyChPtr unsafe.Pointer +} + +func newCond(l sync.Locker) *chCond { + c := &chCond{L: l} + n := make(chan struct{}) + c.notifyChPtr = unsafe.Pointer(&n) + return c +} + +// wait for broadcast calls. Similar to regular sync.Cond +func (c *chCond) wait() { + n := c.notifyChan() + c.L.Unlock() + <-n + c.L.Lock() +} + +// waitWithContext Same as wait() call, but the end condition can also be controlled through the context. +func (c *chCond) waitWithContext(ctx context.Context) bool { + n := c.notifyChan() + c.L.Unlock() + defer c.L.Lock() + select { + case <-n: + return true + case <-ctx.Done(): + return false + default: + return true + } +} + +// broadcast wakes all goroutines waiting on c. +// It is not required for the caller to hold c.L during the call. +func (c *chCond) broadcast() { + n := make(chan struct{}) + ptrOld := atomic.SwapPointer(&c.notifyChPtr, unsafe.Pointer(&n)) + // close old channels to trigger broadcast. + close(*(*chan struct{})(ptrOld)) +} + +func (c *chCond) notifyChan() <-chan struct{} { + ptr := atomic.LoadPointer(&c.notifyChPtr) + return *((*chan struct{})(ptr)) +} diff --git a/pulsar/internal/channel_cond_test.go b/pulsar/internal/channel_cond_test.go new file mode 100644 index 0000000000..a73d44ea26 --- /dev/null +++ b/pulsar/internal/channel_cond_test.go @@ -0,0 +1,55 @@ +// 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 ( + "context" + "sync" + "testing" + "time" +) + +func TestChCond(t *testing.T) { + cond := newCond(&sync.Mutex{}) + wg := sync.WaitGroup{} + wg.Add(1) + go func() { + cond.L.Lock() + cond.wait() + cond.L.Unlock() + wg.Done() + }() + time.Sleep(10 * time.Millisecond) + cond.broadcast() + wg.Wait() +} + +func TestChCondWithContext(t *testing.T) { + cond := newCond(&sync.Mutex{}) + wg := sync.WaitGroup{} + ctx, cancel := context.WithCancel(context.Background()) + wg.Add(1) + go func() { + cond.L.Lock() + cond.waitWithContext(ctx) + cond.L.Unlock() + wg.Done() + }() + cancel() + wg.Wait() +} diff --git a/pulsar/internal/memory_limit_controller.go b/pulsar/internal/memory_limit_controller.go new file mode 100644 index 0000000000..5bf8d59a58 --- /dev/null +++ b/pulsar/internal/memory_limit_controller.go @@ -0,0 +1,101 @@ +// 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 ( + "context" + "sync" + "sync/atomic" +) + +type MemoryLimitController interface { + ReserveMemory(ctx context.Context, size int64) bool + TryReserveMemory(size int64) bool + ForceReserveMemory(size int64) + ReleaseMemory(size int64) + CurrentUsage() int64 + CurrentUsagePercent() float64 + IsMemoryLimited() bool +} + +type memoryLimitController struct { + limit int64 + chCond *chCond + currentUsage int64 +} + +func NewMemoryLimitController(limit int64) MemoryLimitController { + mlc := &memoryLimitController{ + limit: limit, + chCond: newCond(&sync.Mutex{}), + } + return mlc +} + +func (m *memoryLimitController) ReserveMemory(ctx context.Context, size int64) bool { + if !m.TryReserveMemory(size) { + m.chCond.L.Lock() + defer m.chCond.L.Unlock() + + for !m.TryReserveMemory(size) { + if !m.chCond.waitWithContext(ctx) { + return false + } + } + } + return true +} + +func (m *memoryLimitController) TryReserveMemory(size int64) bool { + for { + current := atomic.LoadInt64(&m.currentUsage) + newUsage := current + size + + // This condition means we allowed one request to go over the limit. + if m.IsMemoryLimited() && current > m.limit { + return false + } + + if atomic.CompareAndSwapInt64(&m.currentUsage, current, newUsage) { + return true + } + } +} + +func (m *memoryLimitController) ForceReserveMemory(size int64) { + atomic.AddInt64(&m.currentUsage, size) +} + +func (m *memoryLimitController) ReleaseMemory(size int64) { + newUsage := atomic.AddInt64(&m.currentUsage, -size) + if newUsage+size > m.limit && newUsage <= m.limit { + m.chCond.broadcast() + } +} + +func (m *memoryLimitController) CurrentUsage() int64 { + return atomic.LoadInt64(&m.currentUsage) +} + +func (m *memoryLimitController) CurrentUsagePercent() float64 { + return float64(atomic.LoadInt64(&m.currentUsage)) / float64(m.limit) +} + +func (m *memoryLimitController) IsMemoryLimited() bool { + return m.limit > 0 +} diff --git a/pulsar/internal/memory_limit_controller_test.go b/pulsar/internal/memory_limit_controller_test.go new file mode 100644 index 0000000000..a62c6e6dda --- /dev/null +++ b/pulsar/internal/memory_limit_controller_test.go @@ -0,0 +1,184 @@ +// 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 ( + "context" + "sync" + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestLimit(t *testing.T) { + + mlc := NewMemoryLimitController(100) + + for i := 0; i < 101; i++ { + assert.True(t, mlc.TryReserveMemory(1)) + } + + assert.False(t, mlc.TryReserveMemory(1)) + assert.Equal(t, int64(101), mlc.CurrentUsage()) + assert.InDelta(t, 1.01, mlc.CurrentUsagePercent(), 0.000001) + + mlc.ReleaseMemory(1) + assert.Equal(t, int64(100), mlc.CurrentUsage()) + assert.InDelta(t, 1.0, mlc.CurrentUsagePercent(), 0.000001) + + assert.True(t, mlc.TryReserveMemory(1)) + assert.Equal(t, int64(101), mlc.CurrentUsage()) + + mlc.ForceReserveMemory(99) + assert.False(t, mlc.TryReserveMemory(1)) + assert.Equal(t, int64(200), mlc.CurrentUsage()) + assert.InDelta(t, 2.0, mlc.CurrentUsagePercent(), 0.000001) + + mlc.ReleaseMemory(50) + assert.False(t, mlc.TryReserveMemory(1)) + assert.Equal(t, int64(150), mlc.CurrentUsage()) + assert.InDelta(t, 1.5, mlc.CurrentUsagePercent(), 0.000001) +} + +func TestDisableLimit(t *testing.T) { + mlc := NewMemoryLimitController(-1) + assert.True(t, mlc.TryReserveMemory(1000000)) + assert.True(t, mlc.ReserveMemory(context.Background(), 1000000)) + mlc.ReleaseMemory(1000000) + assert.Equal(t, int64(1000000), mlc.CurrentUsage()) +} + +func TestMultiGoroutineTryReserveMem(t *testing.T) { + mlc := NewMemoryLimitController(10000) + + // Multi goroutine try reserve memory. + wg := sync.WaitGroup{} + + wg.Add(10) + for i := 0; i < 10; i++ { + go func() { + for i := 0; i < 1000; i++ { + assert.True(t, mlc.TryReserveMemory(1)) + } + wg.Done() + }() + } + assert.True(t, mlc.TryReserveMemory(1)) + wg.Wait() + assert.False(t, mlc.TryReserveMemory(1)) + assert.Equal(t, int64(10001), mlc.CurrentUsage()) + assert.InDelta(t, 1.0001, mlc.CurrentUsagePercent(), 0.000001) +} + +func TestReserveWithContext(t *testing.T) { + mlc := NewMemoryLimitController(100) + assert.True(t, mlc.TryReserveMemory(101)) + gorNum := 10 + + // Reserve ctx timeout + waitGroup := sync.WaitGroup{} + waitGroup.Add(gorNum) + ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) + defer cancel() + for i := 0; i < gorNum; i++ { + go func() { + assert.False(t, mlc.ReserveMemory(ctx, 1)) + waitGroup.Done() + }() + } + waitGroup.Wait() + assert.Equal(t, int64(101), mlc.CurrentUsage()) + + // Reserve ctx cancel + waitGroup.Add(gorNum) + cancelCtx, cancel := context.WithCancel(context.Background()) + for i := 0; i < gorNum; i++ { + go func() { + assert.False(t, mlc.ReserveMemory(cancelCtx, 1)) + waitGroup.Done() + }() + } + cancel() + waitGroup.Wait() + assert.Equal(t, int64(101), mlc.CurrentUsage()) +} + +func TestBlocking(t *testing.T) { + mlc := NewMemoryLimitController(100) + assert.True(t, mlc.TryReserveMemory(101)) + assert.Equal(t, int64(101), mlc.CurrentUsage()) + assert.InDelta(t, 1.01, mlc.CurrentUsagePercent(), 0.000001) + + gorNum := 10 + chs := make([]chan int, gorNum) + for i := 0; i < gorNum; i++ { + chs[i] = make(chan int, 1) + go reserveMemory(mlc, chs[i]) + } + + // The threads are blocked since the quota is full + for i := 0; i < gorNum; i++ { + assert.False(t, awaitCh(chs[i])) + } + assert.Equal(t, int64(101), mlc.CurrentUsage()) + + mlc.ReleaseMemory(int64(gorNum)) + for i := 0; i < gorNum; i++ { + assert.True(t, awaitCh(chs[i])) + } + assert.Equal(t, int64(101), mlc.CurrentUsage()) +} + +func TestStepRelease(t *testing.T) { + mlc := NewMemoryLimitController(100) + assert.True(t, mlc.TryReserveMemory(101)) + assert.Equal(t, int64(101), mlc.CurrentUsage()) + assert.InDelta(t, 1.01, mlc.CurrentUsagePercent(), 0.000001) + + gorNum := 10 + ch := make(chan int, 1) + for i := 0; i < gorNum; i++ { + go reserveMemory(mlc, ch) + } + + // The threads are blocked since the quota is full + assert.False(t, awaitCh(ch)) + assert.Equal(t, int64(101), mlc.CurrentUsage()) + + for i := 0; i < gorNum; i++ { + mlc.ReleaseMemory(1) + assert.True(t, awaitCh(ch)) + assert.False(t, awaitCh(ch)) + } + assert.Equal(t, int64(101), mlc.CurrentUsage()) +} + +func reserveMemory(mlc MemoryLimitController, ch chan int) { + mlc.ReserveMemory(context.Background(), 1) + ch <- 1 +} + +func awaitCh(ch chan int) bool { + select { + case <-ch: + return true + case <-time.After(100 * time.Millisecond): + return false + } +} diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 160693cbce..c3a0aa93cd 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -51,13 +51,14 @@ const ( ) 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") - errMetaTooLarge = newError(InvalidMessage, "message metadata size exceeds MaxMessageSize") - errProducerClosed = newError(ProducerClosed, "producer already been closed") + 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") + errMetaTooLarge = newError(InvalidMessage, "message metadata size exceeds MaxMessageSize") + errProducerClosed = newError(ProducerClosed, "producer already been closed") + errMemoryBufferIsFull = newError(ClientMemoryBufferIsFull, "client memory buffer is full") buffersPool sync.Pool ) @@ -483,6 +484,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { // read payload from message uncompressedPayload := msg.Payload + uncompressedPayloadSize := int64(len(uncompressedPayload)) var schemaPayload []byte var err error @@ -494,14 +496,14 @@ func (p *partitionProducer) internalSend(request *sendRequest) { // The block chan must be closed when returned with exception defer request.stopBlock() - if !p.canAddToQueue(request) { + if !p.canAddToQueue(request, uncompressedPayloadSize) { return } if p.options.DisableMultiSchema { if msg.Schema != nil && p.options.Schema != nil && msg.Schema.GetSchemaInfo().hash() != p.options.Schema.GetSchemaInfo().hash() { - p.publishSemaphore.Release() + p.releaseSemaphoreAndMem(uncompressedPayloadSize) request.callback(nil, request.msg, fmt.Errorf("msg schema can not match with producer schema")) p.log.WithError(err).Errorf("The producer %s of the topic %s is disabled the `MultiSchema`", p.producerName, p.topic) return @@ -520,7 +522,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { if uncompressedPayload == nil && schema != nil { schemaPayload, err = schema.Encode(msg.Value) if err != nil { - p.publishSemaphore.Release() + p.releaseSemaphoreAndMem(uncompressedPayloadSize) request.callback(nil, request.msg, newError(SchemaFailure, err.Error())) p.log.WithError(err).Errorf("Schema encode message failed %s", msg.Value) return @@ -536,7 +538,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { if schemaVersion == nil { schemaVersion, err = p.getOrCreateSchema(schema.GetSchemaInfo()) if err != nil { - p.publishSemaphore.Release() + p.releaseSemaphoreAndMem(uncompressedPayloadSize) p.log.WithError(err).Error("get schema version fail") request.callback(nil, request.msg, fmt.Errorf("get schema version fail, err: %w", err)) return @@ -596,7 +598,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { // if msg is too large and chunking is disabled if checkSize > maxMessageSize && !p.options.EnableChunking { - p.publishSemaphore.Release() + p.releaseSemaphoreAndMem(uncompressedPayloadSize) request.callback(nil, request.msg, errMessageTooLarge) p.log.WithError(errMessageTooLarge). WithField("size", checkSize). @@ -615,7 +617,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { } else { payloadChunkSize = int(p._getConn().GetMaxMessageSize()) - proto.Size(mm) if payloadChunkSize <= 0 { - p.publishSemaphore.Release() + p.releaseSemaphoreAndMem(uncompressedPayloadSize) request.callback(nil, msg, errMetaTooLarge) p.log.WithError(errMetaTooLarge). WithField("metadata size", proto.Size(mm)). @@ -663,7 +665,8 @@ func (p *partitionProducer) internalSend(request *sendRequest) { chunkRecorder: cr, } // the permit of first chunk has acquired - if chunkID != 0 && !p.canAddToQueue(nsr) { + if chunkID != 0 && !p.canAddToQueue(nsr, 0) { + p.releaseSemaphoreAndMem(uncompressedPayloadSize - int64(rhs)) return } p.internalSingleSend(mm, compressedPayload[lhs:rhs], nsr, uint32(maxMessageSize)) @@ -688,7 +691,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { // after flushing try again to add the current payload if ok := p.batchBuilder.Add(smm, p.sequenceIDGenerator, uncompressedPayload, request, msg.ReplicationClusters, deliverAt, schemaVersion, multiSchemaEnabled); !ok { - p.publishSemaphore.Release() + p.releaseSemaphoreAndMem(uncompressedPayloadSize) request.callback(nil, request.msg, errFailAddToBatch) p.log.WithField("size", uncompressedSize). WithField("properties", msg.Properties). @@ -797,7 +800,7 @@ func (p *partitionProducer) internalSingleSend(mm *pb.MessageMetadata, maxMessageSize, ); err != nil { request.callback(nil, request.msg, err) - p.publishSemaphore.Release() + p.releaseSemaphoreAndMem(int64(len(msg.Payload))) p.log.WithError(err).Errorf("Single message serialize failed %s", msg.Value) return } @@ -935,7 +938,7 @@ func (p *partitionProducer) failTimeoutMessages() { sr := i.(*sendRequest) if sr.msg != nil { size := len(sr.msg.Payload) - p.publishSemaphore.Release() + p.releaseSemaphoreAndMem(int64(size)) p.metrics.MessagesPending.Dec() p.metrics.BytesPending.Sub(float64(size)) p.metrics.PublishErrorsTimeout.Inc() @@ -1139,8 +1142,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) sr := i.(*sendRequest) if sr.msg != nil { atomic.StoreInt64(&p.lastSequenceID, int64(pi.sequenceID)) - p.publishSemaphore.Release() - + p.releaseSemaphoreAndMem(int64(len(sr.msg.Payload))) p.metrics.PublishLatency.Observe(float64(now-sr.publishTime.UnixNano()) / 1.0e9) p.metrics.MessagesPublished.Inc() p.metrics.MessagesPending.Dec() @@ -1326,7 +1328,12 @@ func (p *partitionProducer) _getConn() internal.Connection { return p.conn.Load().(internal.Connection) } -func (p *partitionProducer) canAddToQueue(sr *sendRequest) bool { +func (p *partitionProducer) releaseSemaphoreAndMem(size int64) { + p.publishSemaphore.Release() + p.client.memLimit.ReleaseMemory(size) +} + +func (p *partitionProducer) canAddToQueue(sr *sendRequest, uncompressedPayloadSize int64) bool { if p.options.DisableBlockIfQueueFull { if !p.publishSemaphore.TryAcquire() { if sr.callback != nil { @@ -1334,9 +1341,24 @@ func (p *partitionProducer) canAddToQueue(sr *sendRequest) bool { } return false } - } else if !p.publishSemaphore.Acquire(sr.ctx) { - sr.callback(nil, sr.msg, errContextExpired) - return false + if !p.client.memLimit.TryReserveMemory(uncompressedPayloadSize) { + p.publishSemaphore.Release() + if sr.callback != nil { + sr.callback(nil, sr.msg, errMemoryBufferIsFull) + } + return false + } + + } else { + if !p.publishSemaphore.Acquire(sr.ctx) { + sr.callback(nil, sr.msg, errContextExpired) + return false + } + if !p.client.memLimit.ReserveMemory(sr.ctx, uncompressedPayloadSize) { + p.publishSemaphore.Release() + sr.callback(nil, sr.msg, errContextExpired) + return false + } } p.metrics.MessagesPending.Inc() p.metrics.BytesPending.Add(float64(len(sr.msg.Payload))) diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index e69a14ce71..f86d01ad9a 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -1779,3 +1779,129 @@ func TestWaitForExclusiveProducer(t *testing.T) { producer1.Close() wg.Wait() } + +func TestMemLimitRejectProducerMessages(t *testing.T) { + + c, err := NewClient(ClientOptions{ + URL: serviceURL, + MemoryLimitBytes: 100 * 1024, + }) + assert.NoError(t, err) + defer c.Close() + + topicName := newTopicName() + producer1, _ := c.CreateProducer(ProducerOptions{ + Topic: topicName, + DisableBlockIfQueueFull: true, + DisableBatching: false, + BatchingMaxPublishDelay: 100 * time.Second, + SendTimeout: 2 * time.Second, + }) + + producer2, _ := c.CreateProducer(ProducerOptions{ + Topic: topicName, + DisableBlockIfQueueFull: true, + DisableBatching: false, + BatchingMaxPublishDelay: 100 * time.Second, + SendTimeout: 2 * time.Second, + }) + + n := 101 + for i := 0; i < n/2; i++ { + producer1.SendAsync(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }, func(id MessageID, message *ProducerMessage, e error) {}) + + producer2.SendAsync(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }, func(id MessageID, message *ProducerMessage, e error) {}) + } + // Last message in order to reach the limit + producer1.SendAsync(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }, func(id MessageID, message *ProducerMessage, e error) {}) + time.Sleep(100 * time.Millisecond) + assert.Equal(t, int64(n*1024), c.(*client).memLimit.CurrentUsage()) + + _, err = producer1.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }) + assert.Error(t, err) + assert.ErrorContains(t, err, getResultStr(ClientMemoryBufferIsFull)) + + _, err = producer2.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }) + assert.Error(t, err) + assert.ErrorContains(t, err, getResultStr(ClientMemoryBufferIsFull)) + + // flush pending msg + err = producer1.Flush() + assert.NoError(t, err) + err = producer2.Flush() + assert.NoError(t, err) + assert.Equal(t, int64(0), c.(*client).memLimit.CurrentUsage()) + + _, err = producer1.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }) + assert.NoError(t, err) + _, err = producer2.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }) + assert.NoError(t, err) +} + +func TestMemLimitContextCancel(t *testing.T) { + + c, err := NewClient(ClientOptions{ + URL: serviceURL, + MemoryLimitBytes: 100 * 1024, + }) + assert.NoError(t, err) + defer c.Close() + + topicName := newTopicName() + producer, _ := c.CreateProducer(ProducerOptions{ + Topic: topicName, + DisableBlockIfQueueFull: false, + DisableBatching: false, + BatchingMaxPublishDelay: 100 * time.Second, + SendTimeout: 2 * time.Second, + }) + + n := 101 + ctx, cancel := context.WithCancel(context.Background()) + for i := 0; i < n; i++ { + producer.SendAsync(ctx, &ProducerMessage{ + Payload: make([]byte, 1024), + }, func(id MessageID, message *ProducerMessage, e error) {}) + } + time.Sleep(100 * time.Millisecond) + assert.Equal(t, int64(n*1024), c.(*client).memLimit.CurrentUsage()) + + wg := sync.WaitGroup{} + wg.Add(1) + go func() { + producer.SendAsync(ctx, &ProducerMessage{ + Payload: make([]byte, 1024), + }, func(id MessageID, message *ProducerMessage, e error) { + assert.Error(t, e) + assert.ErrorContains(t, e, getResultStr(TimeoutError)) + wg.Done() + }) + }() + + // cancel pending msg + cancel() + wg.Wait() + + err = producer.Flush() + assert.NoError(t, err) + assert.Equal(t, int64(0), c.(*client).memLimit.CurrentUsage()) + + _, err = producer.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }) + assert.NoError(t, err) +} From 42ded0d59c46fd3fdaad45f045f7e8bf091131a5 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Thu, 2 Mar 2023 00:04:14 +0800 Subject: [PATCH 022/348] Optimize performance by passing MessageID implementations by pointers (#968) ### Motivation Currently there are three implementations of the `MessageID` interface: - `messageID`: 24 bytes - `trackingMessageID`: 64 bytes - `chunkMessageID`: 80 bytes However, for all methods of them, the receiver is a value rather than a pointer. It's inefficient because each time a method is called, the copy would happen. Reference: https://go.dev/tour/methods/8 ### Modifications - Change the receiver from value to pointer for all `MessageID` implementations. - Use pointers as the returned values and function parameters for these implementations everywhere. The `trackingMessageID.Undefined` method is removed because it's never used now. Though it's a public method, the struct and its factory function are not exposed, so I think it's reasonable. Remove the benchmark added in /~https://github.com/apache/pulsar-client-go/pull/324. The result is obvious and this test is meaningless. I tried passing the `trackingMessageID` by pointer and the result reduced from 8.548 ns/op to 1.628 ns/op. It's obvious because a pointer is only 8 bytes while a `trackingMessageID` is 64 bytes. The overhead of accessing by pointers is far less than copying the extra bytes. --- pulsar/ack_grouping_tracker_test.go | 16 +-- pulsar/consumer_impl.go | 22 ++-- pulsar/consumer_multitopic.go | 16 +-- pulsar/consumer_partition.go | 171 +++++++++++++-------------- pulsar/consumer_partition_test.go | 17 +-- pulsar/consumer_regex.go | 16 +-- pulsar/consumer_test.go | 4 +- pulsar/impl_message.go | 94 +++++++-------- pulsar/impl_message_bench_test.go | 49 -------- pulsar/impl_message_test.go | 14 +-- pulsar/message_chunking_test.go | 4 +- pulsar/negative_acks_tracker.go | 2 +- pulsar/negative_acks_tracker_test.go | 16 +-- pulsar/producer_partition.go | 10 +- pulsar/producer_test.go | 8 +- pulsar/reader_impl.go | 32 ++--- 16 files changed, 216 insertions(+), 275 deletions(-) delete mode 100644 pulsar/impl_message_bench_test.go diff --git a/pulsar/ack_grouping_tracker_test.go b/pulsar/ack_grouping_tracker_test.go index d7903e8f93..e7a67258d6 100644 --- a/pulsar/ack_grouping_tracker_test.go +++ b/pulsar/ack_grouping_tracker_test.go @@ -184,14 +184,14 @@ func TestTimedTrackerCumulativeAck(t *testing.T) { func TestTimedTrackerIsDuplicate(t *testing.T) { tracker := newAckGroupingTracker(nil, func(id MessageID) {}, func(id MessageID) {}) - tracker.add(messageID{batchIdx: 0, batchSize: 3}) - tracker.add(messageID{batchIdx: 2, batchSize: 3}) - assert.True(t, tracker.isDuplicate(messageID{batchIdx: 0, batchSize: 3})) - assert.False(t, tracker.isDuplicate(messageID{batchIdx: 1, batchSize: 3})) - assert.True(t, tracker.isDuplicate(messageID{batchIdx: 2, batchSize: 3})) + tracker.add(&messageID{batchIdx: 0, batchSize: 3}) + tracker.add(&messageID{batchIdx: 2, batchSize: 3}) + assert.True(t, tracker.isDuplicate(&messageID{batchIdx: 0, batchSize: 3})) + assert.False(t, tracker.isDuplicate(&messageID{batchIdx: 1, batchSize: 3})) + assert.True(t, tracker.isDuplicate(&messageID{batchIdx: 2, batchSize: 3})) tracker.flush() - assert.False(t, tracker.isDuplicate(messageID{batchIdx: 0, batchSize: 3})) - assert.False(t, tracker.isDuplicate(messageID{batchIdx: 1, batchSize: 3})) - assert.False(t, tracker.isDuplicate(messageID{batchIdx: 2, batchSize: 3})) + assert.False(t, tracker.isDuplicate(&messageID{batchIdx: 0, batchSize: 3})) + assert.False(t, tracker.isDuplicate(&messageID{batchIdx: 1, batchSize: 3})) + assert.False(t, tracker.isDuplicate(&messageID{batchIdx: 2, batchSize: 3})) } diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index 8ee1822c1b..d16f71967e 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -383,7 +383,7 @@ func (c *consumer) internalTopicSubscribeToPartitions() error { metadata: metadata, subProperties: subProperties, replicateSubscriptionState: c.options.ReplicateSubscriptionState, - startMessageID: trackingMessageID{}, + startMessageID: nil, subscriptionMode: durable, readCompacted: c.options.ReadCompacted, interceptors: c.options.Interceptors, @@ -531,8 +531,8 @@ func (c *consumer) ReconsumeLaterWithCustomProperties(msg Message, customPropert if delay < 0 { delay = 0 } - msgID, ok := c.messageID(msg.ID()) - if !ok { + msgID := c.messageID(msg.ID()) + if msgID == nil { return } props := make(map[string]string) @@ -581,8 +581,8 @@ func (c *consumer) ReconsumeLaterWithCustomProperties(msg Message, customPropert func (c *consumer) Nack(msg Message) { if c.options.EnableDefaultNackBackoffPolicy || c.options.NackBackoffPolicy != nil { - mid, ok := c.messageID(msg.ID()) - if !ok { + mid := c.messageID(msg.ID()) + if mid == nil { return } @@ -743,11 +743,11 @@ func toProtoInitialPosition(p SubscriptionInitialPosition) pb.CommandSubscribe_I return pb.CommandSubscribe_Latest } -func (c *consumer) messageID(msgID MessageID) (trackingMessageID, bool) { - mid, ok := toTrackingMessageID(msgID) - if !ok { +func (c *consumer) messageID(msgID MessageID) *trackingMessageID { + mid := toTrackingMessageID(msgID) + if mid == nil { c.log.Warnf("invalid message id type %T", msgID) - return trackingMessageID{}, false + return nil } partition := int(mid.partitionIdx) @@ -755,10 +755,10 @@ func (c *consumer) messageID(msgID MessageID) (trackingMessageID, bool) { if partition < 0 || partition >= len(c.consumers) { c.log.Warnf("invalid partition index %d expected a partition between [0-%d]", partition, len(c.consumers)) - return trackingMessageID{}, false + return nil } - return mid, true + return mid } func addMessageCryptoIfMissing(client *client, options *ConsumerOptions, topics interface{}) error { diff --git a/pulsar/consumer_multitopic.go b/pulsar/consumer_multitopic.go index 452915a87e..c0fcaefd38 100644 --- a/pulsar/consumer_multitopic.go +++ b/pulsar/consumer_multitopic.go @@ -125,8 +125,8 @@ func (c *multiTopicConsumer) Ack(msg Message) error { // AckID the consumption of a single message, identified by its MessageID func (c *multiTopicConsumer) AckID(msgID MessageID) error { - mid, ok := toTrackingMessageID(msgID) - if !ok { + mid := toTrackingMessageID(msgID) + if mid == nil { c.log.Warnf("invalid message id type %T", msgID) return errors.New("invalid message id type in multi_consumer") } @@ -152,8 +152,8 @@ func (c *multiTopicConsumer) AckCumulative(msg Message) error { // AckIDCumulative the reception of all the messages in the stream up to (and including) // the provided message, identified by its MessageID func (c *multiTopicConsumer) AckIDCumulative(msgID MessageID) error { - mid, ok := toTrackingMessageID(msgID) - if !ok { + mid := toTrackingMessageID(msgID) + if mid == nil { c.log.Warnf("invalid message id type %T", msgID) return errors.New("invalid message id type in multi_consumer") } @@ -203,8 +203,8 @@ func (c *multiTopicConsumer) ReconsumeLaterWithCustomProperties(msg Message, cus func (c *multiTopicConsumer) Nack(msg Message) { if c.options.EnableDefaultNackBackoffPolicy || c.options.NackBackoffPolicy != nil { msgID := msg.ID() - mid, ok := toTrackingMessageID(msgID) - if !ok { + mid := toTrackingMessageID(msgID) + if mid == nil { c.log.Warnf("invalid message id type %T", msgID) return } @@ -221,8 +221,8 @@ func (c *multiTopicConsumer) Nack(msg Message) { } func (c *multiTopicConsumer) NackID(msgID MessageID) { - mid, ok := toTrackingMessageID(msgID) - if !ok { + mid := toTrackingMessageID(msgID) + if mid == nil { c.log.Warnf("invalid message id type %T", msgID) return } diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 95a8d3240d..0f7af3de7d 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -41,10 +41,6 @@ import ( uAtomic "go.uber.org/atomic" ) -var ( - lastestMessageID = LatestMessageID() -) - type consumerState int const ( @@ -98,7 +94,7 @@ type partitionConsumerOpts struct { metadata map[string]string subProperties map[string]string replicateSubscriptionState bool - startMessageID trackingMessageID + startMessageID *trackingMessageID startMessageIDInclusive bool subscriptionMode subscriptionMode readCompacted bool @@ -149,13 +145,13 @@ type partitionConsumer struct { queueSize int32 queueCh chan []*message startMessageID atomicMessageID - lastDequeuedMsg trackingMessageID + lastDequeuedMsg *trackingMessageID eventsCh chan interface{} connectedCh chan struct{} connectClosedCh chan connectionClosed closeCh chan struct{} - clearQueueCh chan func(id trackingMessageID) + clearQueueCh chan func(id *trackingMessageID) nackTracker *negativeAcksTracker dlq *dlqRouter @@ -217,17 +213,17 @@ func (p *availablePermits) reset() { // atomicMessageID is a wrapper for trackingMessageID to make get and set atomic type atomicMessageID struct { - msgID trackingMessageID + msgID *trackingMessageID sync.RWMutex } -func (a *atomicMessageID) get() trackingMessageID { +func (a *atomicMessageID) get() *trackingMessageID { a.RLock() defer a.RUnlock() return a.msgID } -func (a *atomicMessageID) set(msgID trackingMessageID) { +func (a *atomicMessageID) set(msgID *trackingMessageID) { a.Lock() defer a.Unlock() a.msgID = msgID @@ -303,7 +299,7 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon messageCh: messageCh, connectClosedCh: make(chan connectionClosed, 10), closeCh: make(chan struct{}), - clearQueueCh: make(chan func(id trackingMessageID)), + clearQueueCh: make(chan func(id *trackingMessageID)), compressionProviders: sync.Map{}, dlq: dlq, metrics: metrics, @@ -347,7 +343,8 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon pc.log.Info("Created consumer") pc.setConsumerState(consumerReady) - if pc.options.startMessageIDInclusive && pc.startMessageID.get().equal(lastestMessageID.(messageID)) { + startingMessageID := pc.startMessageID.get() + if pc.options.startMessageIDInclusive && startingMessageID != nil && startingMessageID.equal(latestMessageID) { msgID, err := pc.requestGetLastMessageID() if err != nil { pc.nackTracker.Close() @@ -418,10 +415,10 @@ func (pc *partitionConsumer) internalUnsubscribe(unsub *unsubscribeRequest) { pc.setConsumerState(consumerClosed) } -func (pc *partitionConsumer) getLastMessageID() (trackingMessageID, error) { +func (pc *partitionConsumer) getLastMessageID() (*trackingMessageID, error) { if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing { pc.log.WithField("state", state).Error("Failed to redeliver closing or closed consumer") - return trackingMessageID{}, errors.New("failed to redeliver closing or closed consumer") + return nil, errors.New("failed to redeliver closing or closed consumer") } req := &getLastMsgIDRequest{doneCh: make(chan struct{})} pc.eventsCh <- req @@ -436,10 +433,10 @@ func (pc *partitionConsumer) internalGetLastMessageID(req *getLastMsgIDRequest) req.msgID, req.err = pc.requestGetLastMessageID() } -func (pc *partitionConsumer) requestGetLastMessageID() (trackingMessageID, error) { +func (pc *partitionConsumer) requestGetLastMessageID() (*trackingMessageID, error) { if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing { pc.log.WithField("state", state).Error("Failed to getLastMessageID closing or closed consumer") - return trackingMessageID{}, errors.New("failed to getLastMessageID closing or closed consumer") + return nil, errors.New("failed to getLastMessageID closing or closed consumer") } requestID := pc.client.rpcClient.NewRequestID() @@ -451,7 +448,7 @@ func (pc *partitionConsumer) requestGetLastMessageID() (trackingMessageID, error pb.BaseCommand_GET_LAST_MESSAGE_ID, cmdGetLastMessageID) if err != nil { pc.log.WithError(err).Error("Failed to get last message id") - return trackingMessageID{}, err + return nil, err } id := res.Response.GetLastMessageIdResponse.GetLastMessageId() return convertToMessageID(id), nil @@ -463,16 +460,16 @@ func (pc *partitionConsumer) ackID(msgID MessageID, withResponse bool) error { return errors.New("consumer state is closed") } - if cmid, ok := toChunkedMessageID(msgID); ok { + if cmid, ok := msgID.(*chunkMessageID); ok { return pc.unAckChunksTracker.ack(cmid) } - trackingID, ok := toTrackingMessageID(msgID) - if !ok { + trackingID := toTrackingMessageID(msgID) + if trackingID == nil { return errors.New("failed to convert trackingMessageID") } - if !trackingID.Undefined() && trackingID.ack() { + if trackingID != nil && trackingID.ack() { pc.metrics.AcksCounter.Inc() pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9) } else if !pc.options.enableBatchIndexAck { @@ -481,10 +478,10 @@ func (pc *partitionConsumer) ackID(msgID MessageID, withResponse bool) error { var ackReq *ackRequest if withResponse { - ackReq := pc.sendIndividualAck(&trackingID) + ackReq := pc.sendIndividualAck(trackingID) <-ackReq.doneCh } else { - pc.ackGroupingTracker.add(&trackingID) + pc.ackGroupingTracker.add(trackingID) } pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID) if ackReq == nil { @@ -526,15 +523,12 @@ func (pc *partitionConsumer) internalAckIDCumulative(msgID MessageID, withRespon } // chunk message id will be converted to tracking message id - trackingID, ok := toTrackingMessageID(msgID) - if !ok { + trackingID := toTrackingMessageID(msgID) + if trackingID == nil { return errors.New("failed to convert trackingMessageID") } - if trackingID.Undefined() { - return nil - } - var msgIDToAck trackingMessageID + var msgIDToAck *trackingMessageID if trackingID.ackCumulative() || pc.options.enableBatchIndexAck { msgIDToAck = trackingID } else if !trackingID.tracker.hasPrevBatchAcked() { @@ -551,15 +545,15 @@ func (pc *partitionConsumer) internalAckIDCumulative(msgID MessageID, withRespon var ackReq *ackRequest if withResponse { - ackReq := pc.sendCumulativeAck(&msgIDToAck) + ackReq := pc.sendCumulativeAck(msgIDToAck) <-ackReq.doneCh } else { - pc.ackGroupingTracker.addCumulative(&msgIDToAck) + pc.ackGroupingTracker.addCumulative(msgIDToAck) } pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID) - if cmid, ok := toChunkedMessageID(msgID); ok { + if cmid, ok := msgID.(*chunkMessageID); ok { pc.unAckChunksTracker.remove(cmid) } @@ -580,13 +574,13 @@ func (pc *partitionConsumer) sendCumulativeAck(msgID MessageID) *ackRequest { } func (pc *partitionConsumer) NackID(msgID MessageID) { - if cmid, ok := toChunkedMessageID(msgID); ok { + if cmid, ok := msgID.(*chunkMessageID); ok { pc.unAckChunksTracker.nack(cmid) return } - trackingID, ok := toTrackingMessageID(msgID) - if !ok { + trackingID := toTrackingMessageID(msgID) + if trackingID == nil { return } @@ -674,9 +668,9 @@ func (pc *partitionConsumer) Seek(msgID MessageID) error { req := &seekRequest{ doneCh: make(chan struct{}), } - if cmid, ok := toChunkedMessageID(msgID); ok { + if cmid, ok := msgID.(*chunkMessageID); ok { req.msgID = cmid.firstChunkID - } else if tmid, ok := toTrackingMessageID(msgID); ok { + } else if tmid := toTrackingMessageID(msgID); tmid != nil { req.msgID = tmid.messageID } else { // will never reach @@ -695,7 +689,7 @@ func (pc *partitionConsumer) internalSeek(seek *seekRequest) { defer close(seek.doneCh) seek.err = pc.requestSeek(seek.msgID) } -func (pc *partitionConsumer) requestSeek(msgID messageID) error { +func (pc *partitionConsumer) requestSeek(msgID *messageID) error { if err := pc.requestSeekWithoutClear(msgID); err != nil { return err } @@ -703,7 +697,7 @@ func (pc *partitionConsumer) requestSeek(msgID messageID) error { return nil } -func (pc *partitionConsumer) requestSeekWithoutClear(msgID messageID) error { +func (pc *partitionConsumer) requestSeekWithoutClear(msgID *messageID) error { state := pc.getConsumerState() if state == consumerClosing || state == consumerClosed { pc.log.WithField("state", state).Error("failed seek by consumer is closing or has closed") @@ -1063,7 +1057,7 @@ func (pc *partitionConsumer) processMessageChunk(compressedPayload internal.Buff numChunks := msgMeta.GetNumChunksFromMsg() totalChunksSize := int(msgMeta.GetTotalChunkMsgSize()) chunkID := msgMeta.GetChunkId() - msgID := messageID{ + msgID := &messageID{ ledgerID: int64(pbMsgID.GetLedgerId()), entryID: int64(pbMsgID.GetEntryId()), batchIdx: -1, @@ -1105,12 +1099,12 @@ func (pc *partitionConsumer) processMessageChunk(compressedPayload internal.Buff return ctx.chunkedMsgBuffer } -func (pc *partitionConsumer) messageShouldBeDiscarded(msgID trackingMessageID) bool { - if pc.startMessageID.get().Undefined() { +func (pc *partitionConsumer) messageShouldBeDiscarded(msgID *trackingMessageID) bool { + if pc.startMessageID.get() == nil { return false } // if we start at latest message, we should never discard - if pc.options.startMessageID.equal(latestMessageID) { + if pc.options.startMessageID != nil && pc.options.startMessageID.equal(latestMessageID) { return false } @@ -1263,7 +1257,7 @@ func (pc *partitionConsumer) dispatcher() { case clearQueueCb := <-pc.clearQueueCh: // drain the message queue on any new connection by sending a // special nil message to the channel so we know when to stop dropping messages - var nextMessageInQueue trackingMessageID + var nextMessageInQueue *trackingMessageID go func() { pc.queueCh <- nil }() @@ -1272,8 +1266,8 @@ func (pc *partitionConsumer) dispatcher() { // the queue has been drained if m == nil { break - } else if nextMessageInQueue.Undefined() { - nextMessageInQueue, _ = toTrackingMessageID(m[0].msgID) + } else if nextMessageInQueue == nil { + nextMessageInQueue = toTrackingMessageID(m[0].msgID) } } @@ -1311,13 +1305,13 @@ type redeliveryRequest struct { type getLastMsgIDRequest struct { doneCh chan struct{} - msgID trackingMessageID + msgID *trackingMessageID err error } type seekRequest struct { doneCh chan struct{} - msgID messageID + msgID *messageID err error } @@ -1578,15 +1572,15 @@ func (pc *partitionConsumer) grabConn() error { } } -func (pc *partitionConsumer) clearQueueAndGetNextMessage() trackingMessageID { +func (pc *partitionConsumer) clearQueueAndGetNextMessage() *trackingMessageID { if pc.getConsumerState() != consumerReady { - return trackingMessageID{} + return nil } wg := &sync.WaitGroup{} wg.Add(1) - var msgID trackingMessageID + var msgID *trackingMessageID - pc.clearQueueCh <- func(id trackingMessageID) { + pc.clearQueueCh <- func(id *trackingMessageID) { msgID = id wg.Done() } @@ -1599,16 +1593,16 @@ func (pc *partitionConsumer) clearQueueAndGetNextMessage() trackingMessageID { * Clear the internal receiver queue and returns the message id of what was the 1st message in the queue that was * not seen by the application */ -func (pc *partitionConsumer) clearReceiverQueue() trackingMessageID { +func (pc *partitionConsumer) clearReceiverQueue() *trackingMessageID { nextMessageInQueue := pc.clearQueueAndGetNextMessage() - if pc.startMessageID.get().Undefined() { + if pc.startMessageID.get() == nil { return pc.startMessageID.get() } - if !nextMessageInQueue.Undefined() { + if nextMessageInQueue != nil { return getPreviousMessage(nextMessageInQueue) - } else if !pc.lastDequeuedMsg.Undefined() { + } else if pc.lastDequeuedMsg != nil { // If the queue was empty we need to restart from the message just after the last one that has been dequeued // in the past return pc.lastDequeuedMsg @@ -1618,10 +1612,10 @@ func (pc *partitionConsumer) clearReceiverQueue() trackingMessageID { } } -func getPreviousMessage(mid trackingMessageID) trackingMessageID { +func getPreviousMessage(mid *trackingMessageID) *trackingMessageID { if mid.batchIdx >= 0 { - return trackingMessageID{ - messageID: messageID{ + return &trackingMessageID{ + messageID: &messageID{ ledgerID: mid.ledgerID, entryID: mid.entryID, batchIdx: mid.batchIdx - 1, @@ -1634,8 +1628,8 @@ func getPreviousMessage(mid trackingMessageID) trackingMessageID { } // Get on previous message in previous entry - return trackingMessageID{ - messageID: messageID{ + return &trackingMessageID{ + messageID: &messageID{ ledgerID: mid.ledgerID, entryID: mid.entryID - 1, batchIdx: mid.batchIdx, @@ -1734,8 +1728,8 @@ func (pc *partitionConsumer) _getConn() internal.Connection { return pc.conn.Load().(internal.Connection) } -func convertToMessageIDData(msgID trackingMessageID) *pb.MessageIdData { - if msgID.Undefined() { +func convertToMessageIDData(msgID *trackingMessageID) *pb.MessageIdData { + if msgID == nil { return nil } @@ -1745,13 +1739,13 @@ func convertToMessageIDData(msgID trackingMessageID) *pb.MessageIdData { } } -func convertToMessageID(id *pb.MessageIdData) trackingMessageID { +func convertToMessageID(id *pb.MessageIdData) *trackingMessageID { if id == nil { - return trackingMessageID{} + return nil } - msgID := trackingMessageID{ - messageID: messageID{ + msgID := &trackingMessageID{ + messageID: &messageID{ ledgerID: int64(*id.LedgerId), entryID: int64(*id.EntryId), }, @@ -1767,7 +1761,7 @@ type chunkedMsgCtx struct { totalChunks int32 chunkedMsgBuffer internal.Buffer lastChunkedMsgID int32 - chunkedMsgIDs []messageID + chunkedMsgIDs []*messageID receivedTime int64 mu sync.Mutex @@ -1778,12 +1772,12 @@ func newChunkedMsgCtx(numChunksFromMsg int32, totalChunkMsgSize int) *chunkedMsg totalChunks: numChunksFromMsg, chunkedMsgBuffer: internal.NewBuffer(totalChunkMsgSize), lastChunkedMsgID: -1, - chunkedMsgIDs: make([]messageID, numChunksFromMsg), + chunkedMsgIDs: make([]*messageID, numChunksFromMsg), receivedTime: time.Now().Unix(), } } -func (c *chunkedMsgCtx) append(chunkID int32, msgID messageID, partPayload internal.Buffer) { +func (c *chunkedMsgCtx) append(chunkID int32, msgID *messageID, partPayload internal.Buffer) { c.mu.Lock() defer c.mu.Unlock() c.chunkedMsgIDs[chunkID] = msgID @@ -1791,20 +1785,20 @@ func (c *chunkedMsgCtx) append(chunkID int32, msgID messageID, partPayload inter c.lastChunkedMsgID = chunkID } -func (c *chunkedMsgCtx) firstChunkID() messageID { +func (c *chunkedMsgCtx) firstChunkID() *messageID { c.mu.Lock() defer c.mu.Unlock() if len(c.chunkedMsgIDs) == 0 { - return messageID{} + return nil } return c.chunkedMsgIDs[0] } -func (c *chunkedMsgCtx) lastChunkID() messageID { +func (c *chunkedMsgCtx) lastChunkID() *messageID { c.mu.Lock() defer c.mu.Unlock() if len(c.chunkedMsgIDs) == 0 { - return messageID{} + return nil } return c.chunkedMsgIDs[len(c.chunkedMsgIDs)-1] } @@ -1814,9 +1808,13 @@ func (c *chunkedMsgCtx) discard(pc *partitionConsumer) { defer c.mu.Unlock() for _, mid := range c.chunkedMsgIDs { + if mid == nil { + continue + } pc.log.Info("Removing chunk message-id", mid.String()) - tmid, _ := toTrackingMessageID(mid) - pc.AckID(tmid) + if tmid := toTrackingMessageID(mid); tmid != nil { + pc.AckID(tmid) + } } } @@ -1935,40 +1933,41 @@ func (c *chunkedMsgCtxMap) Close() { } type unAckChunksTracker struct { - chunkIDs map[chunkMessageID][]messageID + // TODO: use hash code of chunkMessageID as the key + chunkIDs map[chunkMessageID][]*messageID pc *partitionConsumer mu sync.Mutex } func newUnAckChunksTracker(pc *partitionConsumer) *unAckChunksTracker { return &unAckChunksTracker{ - chunkIDs: make(map[chunkMessageID][]messageID), + chunkIDs: make(map[chunkMessageID][]*messageID), pc: pc, } } -func (u *unAckChunksTracker) add(cmid chunkMessageID, ids []messageID) { +func (u *unAckChunksTracker) add(cmid *chunkMessageID, ids []*messageID) { u.mu.Lock() defer u.mu.Unlock() - u.chunkIDs[cmid] = ids + u.chunkIDs[*cmid] = ids } -func (u *unAckChunksTracker) get(cmid chunkMessageID) []messageID { +func (u *unAckChunksTracker) get(cmid *chunkMessageID) []*messageID { u.mu.Lock() defer u.mu.Unlock() - return u.chunkIDs[cmid] + return u.chunkIDs[*cmid] } -func (u *unAckChunksTracker) remove(cmid chunkMessageID) { +func (u *unAckChunksTracker) remove(cmid *chunkMessageID) { u.mu.Lock() defer u.mu.Unlock() - delete(u.chunkIDs, cmid) + delete(u.chunkIDs, *cmid) } -func (u *unAckChunksTracker) ack(cmid chunkMessageID) error { +func (u *unAckChunksTracker) ack(cmid *chunkMessageID) error { ids := u.get(cmid) for _, id := range ids { if err := u.pc.AckID(id); err != nil { @@ -1979,7 +1978,7 @@ func (u *unAckChunksTracker) ack(cmid chunkMessageID) error { return nil } -func (u *unAckChunksTracker) nack(cmid chunkMessageID) { +func (u *unAckChunksTracker) nack(cmid *chunkMessageID) { ids := u.get(cmid) for _, id := range ids { u.pc.NackID(id) diff --git a/pulsar/consumer_partition_test.go b/pulsar/consumer_partition_test.go index b9a9a02c3f..16c439978b 100644 --- a/pulsar/consumer_partition_test.go +++ b/pulsar/consumer_partition_test.go @@ -48,11 +48,11 @@ func TestSingleMessageIDNoAckTracker(t *testing.T) { // ensure the tracker was set on the message id messages := <-pc.queueCh for _, m := range messages { - assert.Nil(t, m.ID().(trackingMessageID).tracker) + assert.Nil(t, m.ID().(*trackingMessageID).tracker) } // ack the message id - pc.AckID(messages[0].msgID.(trackingMessageID)) + pc.AckID(messages[0].msgID.(*trackingMessageID)) select { case <-eventsCh: @@ -86,11 +86,12 @@ func TestBatchMessageIDNoAckTracker(t *testing.T) { // ensure the tracker was set on the message id messages := <-pc.queueCh for _, m := range messages { - assert.Nil(t, m.ID().(trackingMessageID).tracker) + assert.Nil(t, m.ID().(*trackingMessageID).tracker) } // ack the message id - pc.AckID(messages[0].msgID.(trackingMessageID)) + err := pc.AckID(messages[0].msgID.(*trackingMessageID)) + assert.Nil(t, err) select { case <-eventsCh: @@ -120,12 +121,13 @@ func TestBatchMessageIDWithAckTracker(t *testing.T) { // ensure the tracker was set on the message id messages := <-pc.queueCh for _, m := range messages { - assert.NotNil(t, m.ID().(trackingMessageID).tracker) + assert.NotNil(t, m.ID().(*trackingMessageID).tracker) } // ack all message ids except the last one for i := 0; i < 9; i++ { - pc.AckID(messages[i].msgID.(trackingMessageID)) + err := pc.AckID(messages[i].msgID.(*trackingMessageID)) + assert.Nil(t, err) } select { @@ -135,7 +137,8 @@ func TestBatchMessageIDWithAckTracker(t *testing.T) { } // ack last message - pc.AckID(messages[9].msgID.(trackingMessageID)) + err := pc.AckID(messages[9].msgID.(*trackingMessageID)) + assert.Nil(t, err) select { case <-eventsCh: diff --git a/pulsar/consumer_regex.go b/pulsar/consumer_regex.go index d890c67306..fdfecec076 100644 --- a/pulsar/consumer_regex.go +++ b/pulsar/consumer_regex.go @@ -174,8 +174,8 @@ func (c *regexConsumer) ReconsumeLaterWithCustomProperties(msg Message, customPr // AckID the consumption of a single message, identified by its MessageID func (c *regexConsumer) AckID(msgID MessageID) error { - mid, ok := toTrackingMessageID(msgID) - if !ok { + mid := toTrackingMessageID(msgID) + if mid == nil { c.log.Warnf("invalid message id type %T", msgID) return errors.New("invalid message id type") } @@ -201,8 +201,8 @@ func (c *regexConsumer) AckCumulative(msg Message) error { // AckIDCumulative the reception of all the messages in the stream up to (and including) // the provided message, identified by its MessageID func (c *regexConsumer) AckIDCumulative(msgID MessageID) error { - mid, ok := toTrackingMessageID(msgID) - if !ok { + mid := toTrackingMessageID(msgID) + if mid == nil { c.log.Warnf("invalid message id type %T", msgID) return errors.New("invalid message id type") } @@ -222,8 +222,8 @@ func (c *regexConsumer) AckIDCumulative(msgID MessageID) error { func (c *regexConsumer) Nack(msg Message) { if c.options.EnableDefaultNackBackoffPolicy || c.options.NackBackoffPolicy != nil { msgID := msg.ID() - mid, ok := toTrackingMessageID(msgID) - if !ok { + mid := toTrackingMessageID(msgID) + if mid == nil { c.log.Warnf("invalid message id type %T", msgID) return } @@ -240,8 +240,8 @@ func (c *regexConsumer) Nack(msg Message) { } func (c *regexConsumer) NackID(msgID MessageID) { - mid, ok := toTrackingMessageID(msgID) - if !ok { + mid := toTrackingMessageID(msgID) + if mid == nil { c.log.Warnf("invalid message id type %T", msgID) return } diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index de90c0e7dd..21fa7d04d4 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -910,7 +910,7 @@ func TestConsumerNoBatchCumulativeAck(t *testing.T) { if i == N/2-1 { // cumulative acks the first half of messages - consumer.AckCumulative(msg) + assert.Nil(t, consumer.AckCumulative(msg)) } } @@ -931,7 +931,7 @@ func TestConsumerNoBatchCumulativeAck(t *testing.T) { assert.Nil(t, err) assert.Equal(t, fmt.Sprintf("msg-content-%d", i), string(msg.Payload())) - consumer.Ack(msg) + assert.Nil(t, consumer.Ack(msg)) } } diff --git a/pulsar/impl_message.go b/pulsar/impl_message.go index 39db8e1cb9..68ddecd449 100644 --- a/pulsar/impl_message.go +++ b/pulsar/impl_message.go @@ -39,7 +39,7 @@ type messageID struct { batchSize int32 } -var latestMessageID = messageID{ +var latestMessageID = &messageID{ ledgerID: math.MaxInt64, entryID: math.MaxInt64, batchIdx: -1, @@ -47,7 +47,7 @@ var latestMessageID = messageID{ batchSize: 0, } -var earliestMessageID = messageID{ +var earliestMessageID = &messageID{ ledgerID: -1, entryID: -1, batchIdx: -1, @@ -56,18 +56,14 @@ var earliestMessageID = messageID{ } type trackingMessageID struct { - messageID + *messageID tracker *ackTracker consumer acker receivedTime time.Time } -func (id trackingMessageID) Undefined() bool { - return id == trackingMessageID{} -} - -func (id trackingMessageID) Ack() error { +func (id *trackingMessageID) Ack() error { if id.consumer == nil { return errors.New("consumer is nil in trackingMessageID") } @@ -78,7 +74,7 @@ func (id trackingMessageID) Ack() error { return nil } -func (id trackingMessageID) AckWithResponse() error { +func (id *trackingMessageID) AckWithResponse() error { if id.consumer == nil { return errors.New("consumer is nil in trackingMessageID") } @@ -89,37 +85,37 @@ func (id trackingMessageID) AckWithResponse() error { return nil } -func (id trackingMessageID) Nack() { +func (id *trackingMessageID) Nack() { if id.consumer == nil { return } id.consumer.NackID(id) } -func (id trackingMessageID) NackByMsg(msg Message) { +func (id *trackingMessageID) NackByMsg(msg Message) { if id.consumer == nil { return } id.consumer.NackMsg(msg) } -func (id trackingMessageID) ack() bool { +func (id *trackingMessageID) ack() bool { if id.tracker != nil && id.batchIdx > -1 { return id.tracker.ack(int(id.batchIdx)) } return true } -func (id trackingMessageID) ackCumulative() bool { +func (id *trackingMessageID) ackCumulative() bool { if id.tracker != nil && id.batchIdx > -1 { return id.tracker.ackCumulative(int(id.batchIdx)) } return true } -func (id trackingMessageID) prev() trackingMessageID { - return trackingMessageID{ - messageID: messageID{ +func (id *trackingMessageID) prev() *trackingMessageID { + return &trackingMessageID{ + messageID: &messageID{ ledgerID: id.ledgerID, entryID: id.entryID - 1, partitionIdx: id.partitionIdx, @@ -129,11 +125,11 @@ func (id trackingMessageID) prev() trackingMessageID { } } -func (id messageID) isEntryIDValid() bool { +func (id *messageID) isEntryIDValid() bool { return id.entryID >= 0 } -func (id messageID) greater(other messageID) bool { +func (id *messageID) greater(other *messageID) bool { if id.ledgerID != other.ledgerID { return id.ledgerID > other.ledgerID } @@ -145,17 +141,17 @@ func (id messageID) greater(other messageID) bool { return id.batchIdx > other.batchIdx } -func (id messageID) equal(other messageID) bool { +func (id *messageID) equal(other *messageID) bool { return id.ledgerID == other.ledgerID && id.entryID == other.entryID && id.batchIdx == other.batchIdx } -func (id messageID) greaterEqual(other messageID) bool { +func (id *messageID) greaterEqual(other *messageID) bool { return id.equal(other) || id.greater(other) } -func (id messageID) Serialize() []byte { +func (id *messageID) Serialize() []byte { msgID := &pb.MessageIdData{ LedgerId: proto.Uint64(uint64(id.ledgerID)), EntryId: proto.Uint64(uint64(id.entryID)), @@ -167,27 +163,27 @@ func (id messageID) Serialize() []byte { return data } -func (id messageID) LedgerID() int64 { +func (id *messageID) LedgerID() int64 { return id.ledgerID } -func (id messageID) EntryID() int64 { +func (id *messageID) EntryID() int64 { return id.entryID } -func (id messageID) BatchIdx() int32 { +func (id *messageID) BatchIdx() int32 { return id.batchIdx } -func (id messageID) PartitionIdx() int32 { +func (id *messageID) PartitionIdx() int32 { return id.partitionIdx } -func (id messageID) BatchSize() int32 { +func (id *messageID) BatchSize() int32 { return id.batchSize } -func (id messageID) String() string { +func (id *messageID) String() string { return fmt.Sprintf("%d:%d:%d", id.ledgerID, id.entryID, id.partitionIdx) } @@ -208,7 +204,7 @@ func deserializeMessageID(data []byte) (MessageID, error) { } func newMessageID(ledgerID int64, entryID int64, batchIdx int32, partitionIdx int32, batchSize int32) MessageID { - return messageID{ + return &messageID{ ledgerID: ledgerID, entryID: entryID, batchIdx: batchIdx, @@ -218,9 +214,9 @@ func newMessageID(ledgerID int64, entryID int64, batchIdx int32, partitionIdx in } func newTrackingMessageID(ledgerID int64, entryID int64, batchIdx int32, partitionIdx int32, batchSize int32, - tracker *ackTracker) trackingMessageID { - return trackingMessageID{ - messageID: messageID{ + tracker *ackTracker) *trackingMessageID { + return &trackingMessageID{ + messageID: &messageID{ ledgerID: ledgerID, entryID: entryID, batchIdx: batchIdx, @@ -232,31 +228,23 @@ func newTrackingMessageID(ledgerID int64, entryID int64, batchIdx int32, partiti } } -func toTrackingMessageID(msgID MessageID) (trackingMessageID, bool) { - if mid, ok := msgID.(messageID); ok { - return trackingMessageID{ +func toTrackingMessageID(msgID MessageID) *trackingMessageID { + if mid, ok := msgID.(*messageID); ok { + return &trackingMessageID{ messageID: mid, receivedTime: time.Now(), - }, true - } else if mid, ok := msgID.(trackingMessageID); ok { - return mid, true - } else if cmid, ok := msgID.(chunkMessageID); ok { - return trackingMessageID{ + } + } else if mid, ok := msgID.(*trackingMessageID); ok { + return mid + } else if cmid, ok := msgID.(*chunkMessageID); ok { + return &trackingMessageID{ messageID: cmid.messageID, receivedTime: cmid.receivedTime, consumer: cmid.consumer, - }, true + } } else { - return trackingMessageID{}, false - } -} - -func toChunkedMessageID(msgID MessageID) (chunkMessageID, bool) { - cid, ok := msgID.(chunkMessageID) - if ok { - return cid, true + return nil } - return chunkMessageID{}, false } func timeFromUnixTimestampMillis(timestamp uint64) time.Time { @@ -449,16 +437,16 @@ func (t *ackTracker) toAckSet() []int64 { } type chunkMessageID struct { - messageID + *messageID - firstChunkID messageID + firstChunkID *messageID receivedTime time.Time consumer acker } -func newChunkMessageID(firstChunkID messageID, lastChunkID messageID) chunkMessageID { - return chunkMessageID{ +func newChunkMessageID(firstChunkID *messageID, lastChunkID *messageID) *chunkMessageID { + return &chunkMessageID{ messageID: lastChunkID, firstChunkID: firstChunkID, receivedTime: time.Now(), diff --git a/pulsar/impl_message_bench_test.go b/pulsar/impl_message_bench_test.go deleted file mode 100644 index 4b6ca10cf2..0000000000 --- a/pulsar/impl_message_bench_test.go +++ /dev/null @@ -1,49 +0,0 @@ -// 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 pulsar - -import ( - "testing" -) - -var ( - usedByProducer messageID - usedByConsumer trackingMessageID -) - -func producerCall(id messageID) messageID { - id.entryID++ - return id -} - -func consumerCall(id trackingMessageID) trackingMessageID { - id.entryID++ - return id -} - -func BenchmarkProducerCall(b *testing.B) { - for i := 0; i < b.N; i++ { - usedByProducer = producerCall(usedByProducer) - } -} - -func BenchmarkConsumerCall(b *testing.B) { - for i := 0; i < b.N; i++ { - usedByConsumer = consumerCall(usedByConsumer) - } -} diff --git a/pulsar/impl_message_test.go b/pulsar/impl_message_test.go index 413a39fc37..6a21171ce4 100644 --- a/pulsar/impl_message_test.go +++ b/pulsar/impl_message_test.go @@ -31,11 +31,11 @@ func TestMessageId(t *testing.T) { assert.NoError(t, err) assert.NotNil(t, id2) - assert.Equal(t, int64(1), id2.(messageID).ledgerID) - assert.Equal(t, int64(2), id2.(messageID).entryID) - assert.Equal(t, int32(3), id2.(messageID).batchIdx) - assert.Equal(t, int32(4), id2.(messageID).partitionIdx) - assert.Equal(t, int32(5), id2.(messageID).batchSize) + assert.Equal(t, int64(1), id2.(*messageID).ledgerID) + assert.Equal(t, int64(2), id2.(*messageID).entryID) + assert.Equal(t, int32(3), id2.(*messageID).batchIdx) + assert.Equal(t, int32(4), id2.(*messageID).partitionIdx) + assert.Equal(t, int32(5), id2.(*messageID).batchSize) id, err = DeserializeMessageID(nil) assert.Error(t, err) @@ -110,7 +110,7 @@ func TestAckingMessageIDBatchOne(t *testing.T) { func TestAckingMessageIDBatchTwo(t *testing.T) { tracker := newAckTracker(2) - ids := []trackingMessageID{ + ids := []*trackingMessageID{ newTrackingMessageID(1, 1, 0, 0, 0, tracker), newTrackingMessageID(1, 1, 1, 0, 0, tracker), } @@ -121,7 +121,7 @@ func TestAckingMessageIDBatchTwo(t *testing.T) { // try reverse order tracker = newAckTracker(2) - ids = []trackingMessageID{ + ids = []*trackingMessageID{ newTrackingMessageID(1, 1, 0, 0, 0, tracker), newTrackingMessageID(1, 1, 1, 0, 0, tracker), } diff --git a/pulsar/message_chunking_test.go b/pulsar/message_chunking_test.go index b3d64afaec..aac87c7c13 100644 --- a/pulsar/message_chunking_test.go +++ b/pulsar/message_chunking_test.go @@ -454,10 +454,10 @@ func TestChunkSize(t *testing.T) { }) assert.NoError(t, err) if size <= payloadChunkSize { - _, ok := msgID.(messageID) + _, ok := msgID.(*messageID) assert.Equal(t, true, ok) } else { - _, ok := msgID.(chunkMessageID) + _, ok := msgID.(*chunkMessageID) assert.Equal(t, true, ok) } } diff --git a/pulsar/negative_acks_tracker.go b/pulsar/negative_acks_tracker.go index 79ed6946b3..58f567675b 100644 --- a/pulsar/negative_acks_tracker.go +++ b/pulsar/negative_acks_tracker.go @@ -65,7 +65,7 @@ func newNegativeAcksTracker(rc redeliveryConsumer, delay time.Duration, return t } -func (t *negativeAcksTracker) Add(msgID messageID) { +func (t *negativeAcksTracker) Add(msgID *messageID) { // Always clear up the batch index since we want to track the nack // for the entire batch batchMsgID := messageID{ diff --git a/pulsar/negative_acks_tracker_test.go b/pulsar/negative_acks_tracker_test.go index 5faa9471b6..12d33dee22 100644 --- a/pulsar/negative_acks_tracker_test.go +++ b/pulsar/negative_acks_tracker_test.go @@ -81,13 +81,13 @@ func TestNacksTracker(t *testing.T) { nmc := newNackMockedConsumer(nil) nacks := newNegativeAcksTracker(nmc, testNackDelay, nil, log.DefaultNopLogger()) - nacks.Add(messageID{ + nacks.Add(&messageID{ ledgerID: 1, entryID: 1, batchIdx: 1, }) - nacks.Add(messageID{ + nacks.Add(&messageID{ ledgerID: 2, entryID: 2, batchIdx: 1, @@ -114,25 +114,25 @@ func TestNacksWithBatchesTracker(t *testing.T) { nmc := newNackMockedConsumer(nil) nacks := newNegativeAcksTracker(nmc, testNackDelay, nil, log.DefaultNopLogger()) - nacks.Add(messageID{ + nacks.Add(&messageID{ ledgerID: 1, entryID: 1, batchIdx: 1, }) - nacks.Add(messageID{ + nacks.Add(&messageID{ ledgerID: 1, entryID: 1, batchIdx: 2, }) - nacks.Add(messageID{ + nacks.Add(&messageID{ ledgerID: 1, entryID: 1, batchIdx: 3, }) - nacks.Add(messageID{ + nacks.Add(&messageID{ ledgerID: 2, entryID: 2, batchIdx: 1, @@ -194,7 +194,7 @@ func (msg *mockMessage1) Payload() []byte { } func (msg *mockMessage1) ID() MessageID { - return messageID{ + return &messageID{ ledgerID: 1, entryID: 1, batchIdx: 1, @@ -270,7 +270,7 @@ func (msg *mockMessage2) Payload() []byte { } func (msg *mockMessage2) ID() MessageID { - return messageID{ + return &messageID{ ledgerID: 2, entryID: 2, batchIdx: 1, diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index c3a0aa93cd..744df7995b 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1163,7 +1163,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) if sr.totalChunks > 1 { if sr.chunkID == 0 { sr.chunkRecorder.setFirstChunkID( - messageID{ + &messageID{ int64(response.MessageId.GetLedgerId()), int64(response.MessageId.GetEntryId()), -1, @@ -1172,7 +1172,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) }) } else if sr.chunkID == sr.totalChunks-1 { sr.chunkRecorder.setLastChunkID( - messageID{ + &messageID{ int64(response.MessageId.GetLedgerId()), int64(response.MessageId.GetEntryId()), -1, @@ -1180,7 +1180,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) 0, }) // use chunkMsgID to set msgID - msgID = sr.chunkRecorder.chunkedMsgID + msgID = &sr.chunkRecorder.chunkedMsgID } } @@ -1375,11 +1375,11 @@ func newChunkRecorder() *chunkRecorder { } } -func (c *chunkRecorder) setFirstChunkID(msgID messageID) { +func (c *chunkRecorder) setFirstChunkID(msgID *messageID) { c.chunkedMsgID.firstChunkID = msgID } -func (c *chunkRecorder) setLastChunkID(msgID messageID) { +func (c *chunkRecorder) setLastChunkID(msgID *messageID) { c.chunkedMsgID.messageID = msgID } diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index f86d01ad9a..a6f5e3972c 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -325,7 +325,7 @@ func TestFlushInProducer(t *testing.T) { assert.Nil(t, err) msgCount++ - msgID := msg.ID().(trackingMessageID) + msgID := msg.ID().(*trackingMessageID) // Since messages are batched, they will be sharing the same ledgerId/entryId if ledgerID == -1 { ledgerID = msgID.ledgerID @@ -742,7 +742,7 @@ func TestBatchDelayMessage(t *testing.T) { var delayMsgID int64 ch := make(chan struct{}, 2) producer.SendAsync(ctx, delayMsg, func(id MessageID, producerMessage *ProducerMessage, err error) { - atomic.StoreInt64(&delayMsgID, id.(messageID).entryID) + atomic.StoreInt64(&delayMsgID, id.(*messageID).entryID) ch <- struct{}{} }) delayMsgPublished := false @@ -758,13 +758,13 @@ func TestBatchDelayMessage(t *testing.T) { } var noDelayMsgID int64 producer.SendAsync(ctx, noDelayMsg, func(id MessageID, producerMessage *ProducerMessage, err error) { - atomic.StoreInt64(&noDelayMsgID, id.(messageID).entryID) + atomic.StoreInt64(&noDelayMsgID, id.(*messageID).entryID) }) for i := 0; i < 2; i++ { msg, err := consumer.Receive(context.Background()) assert.Nil(t, err, "unexpected error occurred when recving message from topic") - switch msg.ID().(trackingMessageID).entryID { + switch msg.ID().(*trackingMessageID).entryID { case atomic.LoadInt64(&noDelayMsgID): assert.LessOrEqual(t, time.Since(msg.PublishTime()).Nanoseconds(), int64(batchingDelay*2)) case atomic.LoadInt64(&delayMsgID): diff --git a/pulsar/reader_impl.go b/pulsar/reader_impl.go index 079754b0bf..68dd0843ee 100644 --- a/pulsar/reader_impl.go +++ b/pulsar/reader_impl.go @@ -37,7 +37,7 @@ type reader struct { client *client pc *partitionConsumer messageCh chan ConsumerMessage - lastMessageInBroker trackingMessageID + lastMessageInBroker *trackingMessageID log log.Logger metrics *internal.LeveledMetrics } @@ -51,8 +51,8 @@ func newReader(client *client, options ReaderOptions) (Reader, error) { return nil, newError(InvalidConfiguration, "StartMessageID is required") } - startMessageID, ok := toTrackingMessageID(options.StartMessageID) - if !ok { + startMessageID := toTrackingMessageID(options.StartMessageID) + if startMessageID == nil { // a custom type satisfying MessageID may not be a messageID or trackingMessageID // so re-create messageID using its data deserMsgID, err := deserializeMessageID(options.StartMessageID.Serialize()) @@ -60,8 +60,8 @@ func newReader(client *client, options ReaderOptions) (Reader, error) { return nil, err } // de-serialized MessageID is a messageID - startMessageID = trackingMessageID{ - messageID: deserMsgID.(messageID), + startMessageID = &trackingMessageID{ + messageID: deserMsgID.(*messageID), receivedTime: time.Now(), } } @@ -148,7 +148,7 @@ func (r *reader) Next(ctx context.Context) (Message, error) { // Acknowledge message immediately because the reader is based on non-durable subscription. When it reconnects, // it will specify the subscription position anyway msgID := cm.Message.ID() - if mid, ok := toTrackingMessageID(msgID); ok { + if mid := toTrackingMessageID(msgID); mid != nil { r.pc.lastDequeuedMsg = mid r.pc.AckID(mid) return cm.Message, nil @@ -161,7 +161,7 @@ func (r *reader) Next(ctx context.Context) (Message, error) { } func (r *reader) HasNext() bool { - if !r.lastMessageInBroker.Undefined() && r.hasMoreMessages() { + if r.lastMessageInBroker != nil && r.hasMoreMessages() { return true } @@ -180,7 +180,7 @@ func (r *reader) HasNext() bool { } func (r *reader) hasMoreMessages() bool { - if !r.pc.lastDequeuedMsg.Undefined() { + if r.pc.lastDequeuedMsg != nil { return r.lastMessageInBroker.isEntryIDValid() && r.lastMessageInBroker.greater(r.pc.lastDequeuedMsg.messageID) } @@ -200,29 +200,29 @@ func (r *reader) Close() { r.metrics.ReadersClosed.Inc() } -func (r *reader) messageID(msgID MessageID) (trackingMessageID, bool) { - mid, ok := toTrackingMessageID(msgID) - if !ok { +func (r *reader) messageID(msgID MessageID) *trackingMessageID { + mid := toTrackingMessageID(msgID) + if mid == nil { r.log.Warnf("invalid message id type %T", msgID) - return trackingMessageID{}, false + return nil } partition := int(mid.partitionIdx) // did we receive a valid partition index? if partition < 0 { r.log.Warnf("invalid partition index %d expected", partition) - return trackingMessageID{}, false + return nil } - return mid, true + return mid } func (r *reader) Seek(msgID MessageID) error { r.Lock() defer r.Unlock() - mid, ok := r.messageID(msgID) - if !ok { + mid := r.messageID(msgID) + if mid == nil { return nil } From 73d08862236f75951959827d402d78f74ae12a12 Mon Sep 17 00:00:00 2001 From: tison Date: Thu, 2 Mar 2023 17:28:09 +0800 Subject: [PATCH 023/348] Fix license information for go-rate (#975) go-rate is licensed under GPLv3, which is not compatible with ALv2. It has been already removed in /~https://github.com/apache/pulsar-client-go/pull/799 but we don't update all references. Thank @artursouza for spotting this. Signed-off-by: tison --- distribution/LICENSE.bin.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/distribution/LICENSE.bin.txt b/distribution/LICENSE.bin.txt index e37200faac..79e2c52d28 100644 --- a/distribution/LICENSE.bin.txt +++ b/distribution/LICENSE.bin.txt @@ -218,9 +218,6 @@ MIT License * 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 From 48b7d0195327a81f638a3c51df2010d7eb244bbe Mon Sep 17 00:00:00 2001 From: xiaolong ran Date: Fri, 3 Mar 2023 11:54:38 +0800 Subject: [PATCH 024/348] Fix nack backoff policy logic (#974) Signed-off-by: xiaolongran Motivation Currently, the NackBackoffPolicy does not take effect, because in NackBackoffPolicy, we need to use Msg object for Nack, and MsgId cannot be used for Nack, otherwise the Msg redeliverCount field cannot be obtained for backoff retry. --- pulsar/consumer_impl.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index d16f71967e..7a865740a1 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -587,7 +587,7 @@ func (c *consumer) Nack(msg Message) { } if mid.consumer != nil { - mid.consumer.NackID(msg.ID()) + mid.NackByMsg(msg) return } c.consumers[mid.partitionIdx].NackMsg(msg) From 75d2df3b7d1d1d04fb660a1b6c11ede1d2f161bf Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Fri, 3 Mar 2023 16:27:13 +0800 Subject: [PATCH 025/348] [PIP-165] Auto release idle connections (#963) ### Motivation The go implementation of PIP-165:/~https://github.com/apache/pulsar/issues/15516 ### Modifications * Add new configuration `ConnectionMaxIdleTime` to `ClientOptions` * Add a goroutine to `ConnectionPool` to period check and release idle connections. --- pulsar/client.go | 4 + pulsar/client_impl.go | 14 +++- pulsar/client_impl_test.go | 115 +++++++++++++++++++++++++++++ pulsar/internal/connection.go | 48 ++++++++++++ pulsar/internal/connection_pool.go | 33 ++++++++- pulsar/internal/helper.go | 33 +++++++++ 6 files changed, 244 insertions(+), 3 deletions(-) create mode 100644 pulsar/internal/helper.go diff --git a/pulsar/client.go b/pulsar/client.go index 75b363d7cc..bc3f4f5892 100644 --- a/pulsar/client.go +++ b/pulsar/client.go @@ -143,6 +143,10 @@ type ClientOptions struct { // Default prometheus.DefaultRegisterer MetricsRegisterer prometheus.Registerer + // Release the connection if it is not used for more than ConnectionMaxIdleTime. + // Default is 60 seconds, negative such as -1 to disable. + ConnectionMaxIdleTime time.Duration + EnableTransaction bool // Limit of client memory usage (in byte). The 64M default can guarantee a high producer throughput. diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index 7d90922bf9..7c8fcc9cf4 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -34,6 +34,8 @@ const ( defaultOperationTimeout = 30 * time.Second defaultKeepAliveInterval = 30 * time.Second defaultMemoryLimitBytes = 64 * 1024 * 1024 + defaultConnMaxIdleTime = 180 * time.Second + minConnMaxIdleTime = 60 * time.Second ) type client struct { @@ -56,6 +58,16 @@ func newClient(options ClientOptions) (Client, error) { logger = log.NewLoggerWithLogrus(logrus.StandardLogger()) } + connectionMaxIdleTime := options.ConnectionMaxIdleTime + if connectionMaxIdleTime == 0 { + connectionMaxIdleTime = defaultConnMaxIdleTime + } else if connectionMaxIdleTime > 0 && connectionMaxIdleTime < minConnMaxIdleTime { + return nil, newError(InvalidConfiguration, fmt.Sprintf("Connection max idle time should be at least %f "+ + "seconds", minConnMaxIdleTime.Seconds())) + } else { + logger.Debugf("Disable auto release idle connections") + } + if options.URL == "" { return nil, newError(InvalidConfiguration, "URL is required for client") } @@ -143,7 +155,7 @@ func newClient(options ClientOptions) (Client, error) { c := &client{ cnxPool: internal.NewConnectionPool(tlsConfig, authProvider, connectionTimeout, keepAliveInterval, - maxConnectionsPerHost, logger, metrics), + maxConnectionsPerHost, logger, metrics, connectionMaxIdleTime), log: logger, metrics: metrics, memLimit: internal.NewMemoryLimitController(memLimitBytes), diff --git a/pulsar/client_impl_test.go b/pulsar/client_impl_test.go index b8efe9a631..bb28371f20 100644 --- a/pulsar/client_impl_test.go +++ b/pulsar/client_impl_test.go @@ -21,6 +21,7 @@ import ( "context" "crypto/tls" "fmt" + "log" "net/http" "net/http/httptest" "os" @@ -1123,3 +1124,117 @@ func TestServiceUrlTLSWithTLSTransportWithBasicAuth(t *testing.T) { func TestWebServiceUrlTLSWithTLSTransportWithBasicAuth(t *testing.T) { testTLSTransportWithBasicAuth(t, webServiceURLTLS) } + +func TestConfigureConnectionMaxIdleTime(t *testing.T) { + _, err := NewClient(ClientOptions{ + URL: serviceURL, + ConnectionMaxIdleTime: 1 * time.Second, + }) + + assert.Error(t, err, "Should be failed when the connectionMaxIdleTime is less than minConnMaxIdleTime") + + cli, err := NewClient(ClientOptions{ + URL: serviceURL, + ConnectionMaxIdleTime: -1, // Disabled + }) + + assert.Nil(t, err) + cli.Close() + + cli, err = NewClient(ClientOptions{ + URL: serviceURL, + ConnectionMaxIdleTime: 60 * time.Second, + }) + + assert.Nil(t, err) + cli.Close() +} + +func testSendAndReceive(t *testing.T, producer Producer, consumer Consumer) { + // send 10 messages + for i := 0; i < 10; i++ { + if _, err := producer.Send(context.Background(), &ProducerMessage{ + Payload: []byte(fmt.Sprintf("hello-%d", i)), + }); err != nil { + log.Fatal(err) + } + } + + // receive 10 messages + for i := 0; i < 10; i++ { + msg, err := consumer.Receive(context.Background()) + if err != nil { + log.Fatal(err) + } + + expectMsg := fmt.Sprintf("hello-%d", i) + assert.Equal(t, []byte(expectMsg), msg.Payload()) + // ack message + err = consumer.Ack(msg) + if err != nil { + return + } + } +} + +func TestAutoCloseIdleConnection(t *testing.T) { + cli, err := NewClient(ClientOptions{ + URL: serviceURL, + ConnectionMaxIdleTime: -1, // Disable auto release connections first, we will enable it manually later + }) + + assert.Nil(t, err) + + topic := "TestAutoCloseIdleConnection" + + // create consumer + consumer1, err := cli.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + }) + assert.Nil(t, err) + + // create producer + producer1, err := cli.CreateProducer(ProducerOptions{ + Topic: topic, + DisableBatching: false, + }) + assert.Nil(t, err) + + testSendAndReceive(t, producer1, consumer1) + + pool := cli.(*client).cnxPool + + producer1.Close() + consumer1.Close() + + assert.NotEqual(t, 0, internal.GetConnectionsCount(&pool)) + + internal.StartCleanConnectionsTask(&pool, 2*time.Second) // Enable auto idle connections release manually + + time.Sleep(6 * time.Second) // Need to wait at least 3 * ConnectionMaxIdleTime + + assert.Equal(t, 0, internal.GetConnectionsCount(&pool)) + + // create consumer + consumer2, err := cli.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + }) + assert.Nil(t, err) + + // create producer + producer2, err := cli.CreateProducer(ProducerOptions{ + Topic: topic, + DisableBatching: false, + }) + assert.Nil(t, err) + + // Ensure the client still works + testSendAndReceive(t, producer2, consumer2) + + producer2.Close() + consumer2.Close() + + cli.Close() +} diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 09623e50d8..55d04f3e82 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -170,6 +170,8 @@ type connection struct { metrics *Metrics keepAliveInterval time.Duration + + lastActive time.Time } // connectionOptions defines configurations for creating connection. @@ -927,6 +929,52 @@ func (c *connection) UnregisterListener(id uint64) { delete(c.listeners, id) } +func (c *connection) ResetLastActive() { + c.Lock() + defer c.Unlock() + c.lastActive = time.Now() +} + +func (c *connection) isIdle() bool { + { + c.pendingLock.Lock() + defer c.pendingLock.Unlock() + if len(c.pendingReqs) != 0 { + return false + } + } + + { + c.listenersLock.RLock() + defer c.listenersLock.RUnlock() + if len(c.listeners) != 0 { + return false + } + } + + { + c.consumerHandlersLock.Lock() + defer c.consumerHandlersLock.Unlock() + if len(c.consumerHandlers) != 0 { + return false + } + } + + if len(c.incomingRequestsCh) != 0 || len(c.writeRequestsCh) != 0 { + return false + } + return true +} + +func (c *connection) CheckIdle(maxIdleTime time.Duration) bool { + // We don't need to lock here because this method should only be + // called in a single goroutine of the connectionPool + if !c.isIdle() { + c.lastActive = time.Now() + } + return time.Since(c.lastActive) > maxIdleTime +} + // Close closes the connection by // closing underlying socket connection and closeCh. // This also triggers callbacks to the ConnectionClosed listeners. diff --git a/pulsar/internal/connection_pool.go b/pulsar/internal/connection_pool.go index 4f6d6566fe..bf1e297dad 100644 --- a/pulsar/internal/connection_pool.go +++ b/pulsar/internal/connection_pool.go @@ -47,6 +47,7 @@ type connectionPool struct { maxConnectionsPerHost int32 roundRobinCnt int32 keepAliveInterval time.Duration + closeCh chan struct{} metrics *Metrics log log.Logger @@ -60,8 +61,9 @@ func NewConnectionPool( keepAliveInterval time.Duration, maxConnectionsPerHost int, logger log.Logger, - metrics *Metrics) ConnectionPool { - return &connectionPool{ + metrics *Metrics, + connectionMaxIdleTime time.Duration) ConnectionPool { + p := &connectionPool{ connections: make(map[string]*connection), tlsOptions: tlsOptions, auth: auth, @@ -70,7 +72,10 @@ func NewConnectionPool( keepAliveInterval: keepAliveInterval, log: logger, metrics: metrics, + closeCh: make(chan struct{}), } + go p.checkAndCleanIdleConnections(connectionMaxIdleTime) + return p } func (p *connectionPool) GetConnection(logicalAddr *url.URL, physicalAddr *url.URL) (Connection, error) { @@ -109,6 +114,7 @@ func (p *connectionPool) GetConnection(logicalAddr *url.URL, physicalAddr *url.U p.Unlock() conn.start() } else { + conn.ResetLastActive() // we already have a connection p.Unlock() } @@ -119,6 +125,7 @@ func (p *connectionPool) GetConnection(logicalAddr *url.URL, physicalAddr *url.U func (p *connectionPool) Close() { p.Lock() + close(p.closeCh) for k, c := range p.connections { delete(p.connections, k) c.Close() @@ -134,3 +141,25 @@ func (p *connectionPool) getMapKey(addr *url.URL) string { idx := cnt % p.maxConnectionsPerHost return fmt.Sprint(addr.Host, '-', idx) } + +func (p *connectionPool) checkAndCleanIdleConnections(maxIdleTime time.Duration) { + if maxIdleTime < 0 { + return + } + for { + select { + case <-p.closeCh: + return + case <-time.After(maxIdleTime): + p.Lock() + for k, c := range p.connections { + if c.CheckIdle(maxIdleTime) { + c.log.Debugf("Closed connection due to inactivity.") + delete(p.connections, k) + c.Close() + } + } + p.Unlock() + } + } +} diff --git a/pulsar/internal/helper.go b/pulsar/internal/helper.go new file mode 100644 index 0000000000..3bca1ee006 --- /dev/null +++ b/pulsar/internal/helper.go @@ -0,0 +1,33 @@ +// 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 "time" + +// These method should only be used by tests + +func StartCleanConnectionsTask(p *ConnectionPool, connectionMaxIdleTime time.Duration) { + go (*p).(*connectionPool).checkAndCleanIdleConnections(connectionMaxIdleTime) +} + +func GetConnectionsCount(p *ConnectionPool) int { + pool := (*p).(*connectionPool) + pool.Lock() + defer pool.Unlock() + return len(pool.connections) +} From c5956779f078176b6f915892dd47d9cbdb0d0fea Mon Sep 17 00:00:00 2001 From: Masahiro Sakamoto Date: Fri, 3 Mar 2023 18:51:05 +0900 Subject: [PATCH 026/348] Athenz auth provider supports Copper Argos (#960) --- pulsar/auth/athenz.go | 177 +++++++++++++++++++++++++------------ pulsar/auth/athenz_test.go | 145 +++++++++++++++++++++++++++--- 2 files changed, 257 insertions(+), 65 deletions(-) diff --git a/pulsar/auth/athenz.go b/pulsar/auth/athenz.go index f490be95ac..35b3cc4e3d 100644 --- a/pulsar/auth/athenz.go +++ b/pulsar/auth/athenz.go @@ -32,28 +32,33 @@ import ( ) const ( - minExpire = 2 * time.Hour - maxExpire = 24 * time.Hour - AthenzRoleAuthHeader = "Athenz-Role-Auth" + minExpire = 2 * time.Hour + maxExpire = 24 * time.Hour + defaultKeyID = "0" + defaultRoleHeader = "Athenz-Role-Auth" ) type athenzAuthProvider struct { - providerDomain string - tenantDomain string - tenantService string - privateKey string - keyID string - principalHeader string - ztsURL string - tokenBuilder zms.TokenBuilder - roleToken zts.RoleToken - zmsNewTokenBuilder func(domain, name string, privateKeyPEM []byte, keyVersion string) (zms.TokenBuilder, error) - ztsNewRoleToken func(tok zms.Token, domain string, opts zts.RoleTokenOptions) zts.RoleToken + providerDomain string + tenantDomain string + tenantService string + privateKey string + keyID string + x509CertChain string + caCert string + principalHeader string + roleHeader string + ztsURL string + tokenBuilder zms.TokenBuilder + roleToken zts.RoleToken + zmsNewTokenBuilder func(domain, name string, privateKeyPEM []byte, keyVersion string) (zms.TokenBuilder, error) + ztsNewRoleToken func(tok zms.Token, domain string, opts zts.RoleTokenOptions) zts.RoleToken + ztsNewRoleTokenFromCert func(certFile, keyFile, domain string, opts zts.RoleTokenOptions) zts.RoleToken T http.RoundTripper } -type privateKeyURI struct { +type parsedURI struct { Scheme string MediaTypeAndEncodingType string Data string @@ -67,7 +72,10 @@ func NewAuthenticationAthenzWithParams(params map[string]string) (Provider, erro params["tenantService"], params["privateKey"], params["keyId"], + params["x509CertChain"], + params["caCert"], params["principalHeader"], + params["roleHeader"], params["ztsUrl"], ), nil } @@ -78,68 +86,97 @@ func NewAuthenticationAthenz( tenantService string, privateKey string, keyID string, + x509CertChain string, + caCert string, principalHeader string, + roleHeader string, ztsURL string) Provider { - var fixedKeyID string - if keyID == "" { - fixedKeyID = "0" - } else { + fixedKeyID := defaultKeyID + if keyID != "" { fixedKeyID = keyID } + + fixedRoleHeader := defaultRoleHeader + if roleHeader != "" { + fixedRoleHeader = roleHeader + } + ztsNewRoleToken := func(tok zms.Token, domain string, opts zts.RoleTokenOptions) zts.RoleToken { return zts.RoleToken(zts.NewRoleToken(tok, domain, opts)) } + ztsNewRoleTokenFromCert := func(certFile, keyFile, domain string, opts zts.RoleTokenOptions) zts.RoleToken { + return zts.RoleToken(zts.NewRoleTokenFromCert(certFile, keyFile, domain, opts)) + } + return &athenzAuthProvider{ - providerDomain: providerDomain, - tenantDomain: tenantDomain, - tenantService: tenantService, - privateKey: privateKey, - keyID: fixedKeyID, - principalHeader: principalHeader, - ztsURL: strings.TrimSuffix(ztsURL, "/"), - zmsNewTokenBuilder: zms.NewTokenBuilder, - ztsNewRoleToken: ztsNewRoleToken, + providerDomain: providerDomain, + tenantDomain: tenantDomain, + tenantService: tenantService, + privateKey: privateKey, + keyID: fixedKeyID, + x509CertChain: x509CertChain, + caCert: caCert, + principalHeader: principalHeader, + roleHeader: fixedRoleHeader, + ztsURL: strings.TrimSuffix(ztsURL, "/"), + zmsNewTokenBuilder: zms.NewTokenBuilder, + ztsNewRoleToken: ztsNewRoleToken, + ztsNewRoleTokenFromCert: ztsNewRoleTokenFromCert, } } func (p *athenzAuthProvider) Init() error { - uriSt := parseURI(p.privateKey) - var keyData []byte + if p.providerDomain == "" || p.privateKey == "" || p.ztsURL == "" { + return errors.New("missing required parameters") + } - if uriSt.Scheme == "data" { - if uriSt.MediaTypeAndEncodingType != "application/x-pem-file;base64" { - return errors.New("Unsupported mediaType or encodingType: " + uriSt.MediaTypeAndEncodingType) + var roleToken zts.RoleToken + opts := zts.RoleTokenOptions{ + BaseZTSURL: p.ztsURL + "/zts/v1", + MinExpire: minExpire, + MaxExpire: maxExpire, + AuthHeader: p.principalHeader, + } + + if p.x509CertChain != "" { + // use Copper Argos + certURISt := parseURI(p.x509CertChain) + keyURISt := parseURI(p.privateKey) + + if certURISt.Scheme != "file" || keyURISt.Scheme != "file" { + return errors.New("x509CertChain and privateKey must be specified as file paths") + } + + if p.caCert != "" { + caCertData, err := loadPEM(p.caCert) + if err != nil { + return err + } + opts.CACert = caCertData } - key, err := base64.StdEncoding.DecodeString(uriSt.Data) + + roleToken = p.ztsNewRoleTokenFromCert(certURISt.Path, keyURISt.Path, p.providerDomain, opts) + } else { + if p.tenantDomain == "" || p.tenantService == "" { + return errors.New("missing required parameters") + } + + keyData, err := loadPEM(p.privateKey) if err != nil { return err } - keyData = key - } else if uriSt.Scheme == "file" { - key, err := os.ReadFile(uriSt.Path) + + tb, err := p.zmsNewTokenBuilder(p.tenantDomain, p.tenantService, keyData, p.keyID) if err != nil { return err } - keyData = key - } else { - return errors.New("Unsupported URI Scheme: " + uriSt.Scheme) - } + p.tokenBuilder = tb - tb, err := p.zmsNewTokenBuilder(p.tenantDomain, p.tenantService, keyData, p.keyID) - if err != nil { - return err + roleToken = p.ztsNewRoleToken(p.tokenBuilder.Token(), p.providerDomain, opts) } - p.tokenBuilder = tb - roleToken := p.ztsNewRoleToken(p.tokenBuilder.Token(), p.providerDomain, zts.RoleTokenOptions{ - BaseZTSURL: p.ztsURL + "/zts/v1", - MinExpire: minExpire, - MaxExpire: maxExpire, - AuthHeader: p.principalHeader, - }) p.roleToken = roleToken - return nil } @@ -164,8 +201,8 @@ func (p *athenzAuthProvider) Close() error { return nil } -func parseURI(uri string) privateKeyURI { - var uriSt privateKeyURI +func parseURI(uri string) parsedURI { + var uriSt parsedURI // scheme mediatype[;base64] path file const expression = `^(?:([^:/?#]+):)(?:([;/\\\-\w]*),)?(?:/{0,2}((?:[^?#/]*/)*))?([^?#]*)` @@ -177,17 +214,47 @@ func parseURI(uri string) privateKeyURI { uriSt.MediaTypeAndEncodingType = groups[2] uriSt.Data = groups[4] uriSt.Path = groups[3] + groups[4] + } else { + // consider a file path specified instead of a URI + uriSt.Scheme = "file" + uriSt.Path = uri } return uriSt } +func loadPEM(uri string) ([]byte, error) { + uriSt := parseURI(uri) + var pemData []byte + + if uriSt.Scheme == "data" { + if uriSt.MediaTypeAndEncodingType != "application/x-pem-file;base64" { + return nil, errors.New("Unsupported mediaType or encodingType: " + uriSt.MediaTypeAndEncodingType) + } + pem, err := base64.StdEncoding.DecodeString(uriSt.Data) + if err != nil { + return nil, err + } + pemData = pem + } else if uriSt.Scheme == "file" { + pem, err := os.ReadFile(uriSt.Path) + if err != nil { + return nil, err + } + pemData = pem + } else { + return nil, errors.New("Unsupported URI Scheme: " + uriSt.Scheme) + } + + return pemData, nil +} + func (p *athenzAuthProvider) RoundTrip(req *http.Request) (*http.Response, error) { tok, err := p.roleToken.RoleTokenValue() if err != nil { return nil, err } - req.Header.Add(AthenzRoleAuthHeader, tok) + req.Header.Add(p.roleHeader, tok) return p.T.RoundTrip(req) } diff --git a/pulsar/auth/athenz_test.go b/pulsar/auth/athenz_test.go index 8541ca98f5..89118081f9 100644 --- a/pulsar/auth/athenz_test.go +++ b/pulsar/auth/athenz_test.go @@ -30,7 +30,11 @@ import ( "github.com/stretchr/testify/mock" ) -const tlsClientKeyPath = "../../integration-tests/certs/client-key.pem" +const ( + clientKeyPath = "../../integration-tests/certs/client-key.pem" + clientCertPath = "../../integration-tests/certs/client-cert.pem" + caCertPath = "../../integration-tests/certs/cacert.pem" +) type MockTokenBuilder struct { mock.Mock @@ -69,7 +73,7 @@ func (m *MockRoleToken) RoleTokenValue() (string, error) { func MockZmsNewTokenBuilder(domain, name string, privateKeyPEM []byte, keyVersion string) (zms.TokenBuilder, error) { // assertion - key, err := os.ReadFile(tlsClientKeyPath) + key, err := os.ReadFile(clientKeyPath) if err != nil { return nil, err } @@ -105,16 +109,35 @@ func MockZtsNewRoleToken(tok zms.Token, domain string, opts zts.RoleTokenOptions return mockRoleToken } +func MockZtsNewRoleTokenFromCert(certFile, keyFile, domain string, opts zts.RoleTokenOptions) zts.RoleToken { + // assertion + if certFile != clientCertPath || + keyFile != clientKeyPath || + domain != "pulsar.test.provider" || + opts.BaseZTSURL != "http://localhost:9999/zts/v1" || + opts.AuthHeader != "" || + opts.CACert == nil { + return nil + } + + mockRoleToken := new(MockRoleToken) + mockRoleToken.On("RoleTokenValue").Return("mockRoleTokenFromCert", nil) + return mockRoleToken +} + func TestAthenzAuth(t *testing.T) { - privateKey := "file://" + tlsClientKeyPath + privateKey := "file://" + clientKeyPath provider := NewAuthenticationAthenz( - "pulsar.test.provider", - "pulsar.test.tenant", - "service", - privateKey, - "", - "", - "http://localhost:9999") + "pulsar.test.provider", // providerDomain + "pulsar.test.tenant", // tenantDomain + "service", // tenantService + privateKey, // privateKey + "", // keyID + "", // x509CertChain + "", // caCert + "", // principalHeader + "", // roleHeader + "http://localhost:9999") // ztsURL // inject mock function athenz := provider.(*athenzAuthProvider) @@ -128,3 +151,105 @@ func TestAthenzAuth(t *testing.T) { assert.Equal(t, []byte("mockRoleToken"), data) assert.NoError(t, err) } + +func TestCopperArgos(t *testing.T) { + privateKey := "file://" + clientKeyPath + x509CertChain := "file://" + clientCertPath + caCert := "file://" + caCertPath + + provider := NewAuthenticationAthenz( + "pulsar.test.provider", // providerDomain + "", // tenantDomain + "", // tenantService + privateKey, // privateKey + "", // keyID + x509CertChain, // x509CertChain + caCert, // caCert + "", // principalHeader + "", // roleHeader + "http://localhost:9999") // ztsURL + + // inject mock function + athenz := provider.(*athenzAuthProvider) + athenz.ztsNewRoleTokenFromCert = MockZtsNewRoleTokenFromCert + + err := athenz.Init() + assert.NoError(t, err) + + data, err := athenz.GetData() + assert.Equal(t, []byte("mockRoleTokenFromCert"), data) + assert.NoError(t, err) +} + +func TestIllegalParams(t *testing.T) { + privateKey := "file://" + clientKeyPath + x509CertChain := "file://" + clientCertPath + + provider := NewAuthenticationAthenz( + "pulsar.test.provider", // providerDomain + "", // tenantDomain + "", // tenantService + "", // privateKey + "", // keyID + "", // x509CertChain + "", // caCert + "", // principalHeader + "", // roleHeader + "http://localhost:9999") // ztsURL + athenz := provider.(*athenzAuthProvider) + + err := athenz.Init() + assert.Error(t, err, "Should fail due to missing privateKey parameter") + assert.Equal(t, "missing required parameters", err.Error()) + + provider = NewAuthenticationAthenz( + "pulsar.test.provider", // providerDomain + "pulsar.test.tenant", // tenantDomain + "", // tenantService + privateKey, // privateKey + "", // keyID + "", // x509CertChain + "", // caCert + "", // principalHeader + "", // roleHeader + "http://localhost:9999") // ztsURL + athenz = provider.(*athenzAuthProvider) + + err = athenz.Init() + assert.Error(t, err, "Should fail due to missing tenantService parameter") + assert.Equal(t, "missing required parameters", err.Error()) + + provider = NewAuthenticationAthenz( + "pulsar.test.provider", // providerDomain + "", // tenantDomain + "", // tenantService + privateKey, // privateKey + "", // keyID + "data:foo", // x509CertChain + "", // caCert + "", // principalHeader + "", // roleHeader + "http://localhost:9999") // ztsURL + athenz = provider.(*athenzAuthProvider) + + err = athenz.Init() + assert.Error(t, err, "Should fail due to incorrect x509CertChain scheme") + assert.Equal(t, "x509CertChain and privateKey must be specified as file paths", err.Error()) + + provider = NewAuthenticationAthenz( + "pulsar.test.provider", // providerDomain + "", // tenantDomain + "", // tenantService + "data:bar", // privateKey + "", // keyID + x509CertChain, // x509CertChain + "", // caCert + "", // principalHeader + "", // roleHeader + "http://localhost:9999") // ztsURL + athenz = provider.(*athenzAuthProvider) + + err = athenz.Init() + assert.Error(t, err, "Should fail due to incorrect privateKey scheme") + assert.Equal(t, "x509CertChain and privateKey must be specified as file paths", err.Error()) +} From 96bc58fd5e7ffb041703a082f81052a188f9c8ba Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Tue, 7 Mar 2023 17:15:00 +0800 Subject: [PATCH 027/348] [fix] Fix data race in checkAndCleanIdleConnections (#981) ### Motivation From the data race log of the issue: ``` 2023-03-07T02:05:50.0096902Z WARNING: DATA RACE 2023-03-07T02:05:50.0097176Z Read at 0x00c0003bb298 by goroutine 111: 2023-03-07T02:05:50.0097767Z github.com/apache/pulsar-client-go/pulsar/internal.(*connectionPool).checkAndCleanIdleConnections() 2023-03-07T02:05:50.0098468Z /pulsar/pulsar-client-go/pulsar/internal/connection_pool.go:157 +0x24d 2023-03-07T02:05:50.0099058Z github.com/apache/pulsar-client-go/pulsar/internal.StartCleanConnectionsTask.func1() 2023-03-07T02:05:50.0099727Z /pulsar/pulsar-client-go/pulsar/internal/helper.go:25 +0x47 2023-03-07T02:05:50.0099934Z 2023-03-07T02:05:50.0100066Z Previous write at 0x00c0003bb298 by goroutine 69: 2023-03-07T02:05:50.0100537Z github.com/apache/pulsar-client-go/pulsar/internal.(*connection).connect() 2023-03-07T02:05:50.0101126Z /pulsar/pulsar-client-go/pulsar/internal/connection.go:277 +0x5d2 2023-03-07T02:05:50.0101649Z github.com/apache/pulsar-client-go/pulsar/internal.(*connection).start.func1() 2023-03-07T02:05:50.0102222Z /pulsar/pulsar-client-go/pulsar/internal/connection.go:230 +0x34 ``` We can see that the read access of c.log here: /~https://github.com/apache/pulsar-client-go/blob/c5956779f078176b6f915892dd47d9cbdb0d0fea/pulsar/internal/connection_pool.go#L157 is conflict with the write access of c.log here: /~https://github.com/apache/pulsar-client-go/blob/c5956779f078176b6f915892dd47d9cbdb0d0fea/pulsar/internal/connection.go#L277 This caused the data race. ### Modifications * Using p.log to avoid accessing the c.log --- pulsar/internal/connection_pool.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar/internal/connection_pool.go b/pulsar/internal/connection_pool.go index bf1e297dad..6ff7991992 100644 --- a/pulsar/internal/connection_pool.go +++ b/pulsar/internal/connection_pool.go @@ -154,7 +154,8 @@ func (p *connectionPool) checkAndCleanIdleConnections(maxIdleTime time.Duration) p.Lock() for k, c := range p.connections { if c.CheckIdle(maxIdleTime) { - c.log.Debugf("Closed connection due to inactivity.") + p.log.Debugf("Closed connection from pool due to inactivity. logical_addr=%+v physical_addr=%+v", + c.logicalAddr, c.physicalAddr) delete(p.connections, k) c.Close() } From d98c4f17c6f8927072d146f4a10c8df73e21be6e Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Tue, 7 Mar 2023 18:14:11 +0800 Subject: [PATCH 028/348] [refactor]: Refactor the toTrackingMessageID() (#972) ### Motivation `toTrackingMessageID()` is a function that is widely used in `consumer` implementation. It can convert an interface type `MessageID` into an struct type `trackingMessageID`. In addition, the second return value also plays a role in checking the `MessageID` type. In other words, it indicates that `MessageID` **cannot** be a user-defined type. From the perspective of code readability, `toTrackingMessageID()` should not do both. **Note**: After #968 , `toTrackingMessageID()` returns only a pointer now. The role of original `ok` is replaced by nil pointer now. However, the main content discussed in this PR has not changed. For example. /~https://github.com/apache/pulsar-client-go/blob/e2ea255052e8a527091791ef368851d885ee2d45/pulsar/consumer_regex.go#L176-L181 This example is the correct usage. The `ok` returned by `toTrackingMessageID()` is used to reject user-defined `MessageID`. /~https://github.com/apache/pulsar-client-go/blob/e2ea255052e8a527091791ef368851d885ee2d45/pulsar/consumer_partition.go#L470-L473 This example is a bit vague. The actual effect here is the same as the previous example. But it return an error `failed to convert trackingMessageID` which is confusing. /~https://github.com/apache/pulsar-client-go/blob/e2ea255052e8a527091791ef368851d885ee2d45/pulsar/consumer_partition.go#L1816-L1820 In this case. We just want to convert `MessageID` into `trackingMessageID`. We do not care what it really is because it's not possible an invalid `MessageID` implementation. So, original `toTrackingMessageID()` needs to require a careful look to use it correctly. I think it would be better to split it into two different method. `toTrackingMessageID()` just do the struct conversion, which it's more clearly. And when the new messageID type is added, we can just modify the `checkMessageIDType`. ### Modifications - Refactor the `toTrackingMessageID()` - Add the `checkMessageIDType()` to check whether `MessageID` is user-defined. --- pulsar/consumer_impl.go | 15 ++++++++---- pulsar/consumer_multitopic.go | 16 ++++++------- pulsar/consumer_partition.go | 44 ++++++++++++++++++++++++----------- pulsar/consumer_regex.go | 23 ++++++++++-------- pulsar/impl_message.go | 44 +++++++++++++++++++++++------------ pulsar/reader_impl.go | 30 +++++++++++------------- 6 files changed, 106 insertions(+), 66 deletions(-) diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index 7a865740a1..d19e52237f 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -531,10 +531,17 @@ func (c *consumer) ReconsumeLaterWithCustomProperties(msg Message, customPropert if delay < 0 { delay = 0 } + + if !checkMessageIDType(msg.ID()) { + c.log.Warnf("invalid message id type %T", msg.ID()) + return + } + msgID := c.messageID(msg.ID()) if msgID == nil { return } + props := make(map[string]string) for k, v := range msg.Properties() { props[k] = v @@ -580,6 +587,10 @@ func (c *consumer) ReconsumeLaterWithCustomProperties(msg Message, customPropert } func (c *consumer) Nack(msg Message) { + if !checkMessageIDType(msg.ID()) { + c.log.Warnf("invalid message id type %T", msg.ID()) + return + } if c.options.EnableDefaultNackBackoffPolicy || c.options.NackBackoffPolicy != nil { mid := c.messageID(msg.ID()) if mid == nil { @@ -745,10 +756,6 @@ func toProtoInitialPosition(p SubscriptionInitialPosition) pb.CommandSubscribe_I func (c *consumer) messageID(msgID MessageID) *trackingMessageID { mid := toTrackingMessageID(msgID) - if mid == nil { - c.log.Warnf("invalid message id type %T", msgID) - return nil - } partition := int(mid.partitionIdx) // did we receive a valid partition index? diff --git a/pulsar/consumer_multitopic.go b/pulsar/consumer_multitopic.go index c0fcaefd38..8108c294b5 100644 --- a/pulsar/consumer_multitopic.go +++ b/pulsar/consumer_multitopic.go @@ -125,11 +125,11 @@ func (c *multiTopicConsumer) Ack(msg Message) error { // AckID the consumption of a single message, identified by its MessageID func (c *multiTopicConsumer) AckID(msgID MessageID) error { - mid := toTrackingMessageID(msgID) - if mid == nil { + if !checkMessageIDType(msgID) { c.log.Warnf("invalid message id type %T", msgID) return errors.New("invalid message id type in multi_consumer") } + mid := toTrackingMessageID(msgID) if mid.consumer == nil { c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID) @@ -152,11 +152,11 @@ func (c *multiTopicConsumer) AckCumulative(msg Message) error { // AckIDCumulative the reception of all the messages in the stream up to (and including) // the provided message, identified by its MessageID func (c *multiTopicConsumer) AckIDCumulative(msgID MessageID) error { - mid := toTrackingMessageID(msgID) - if mid == nil { + if !checkMessageIDType(msgID) { c.log.Warnf("invalid message id type %T", msgID) return errors.New("invalid message id type in multi_consumer") } + mid := toTrackingMessageID(msgID) if mid.consumer == nil { c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID) @@ -203,11 +203,11 @@ func (c *multiTopicConsumer) ReconsumeLaterWithCustomProperties(msg Message, cus func (c *multiTopicConsumer) Nack(msg Message) { if c.options.EnableDefaultNackBackoffPolicy || c.options.NackBackoffPolicy != nil { msgID := msg.ID() - mid := toTrackingMessageID(msgID) - if mid == nil { + if !checkMessageIDType(msgID) { c.log.Warnf("invalid message id type %T", msgID) return } + mid := toTrackingMessageID(msgID) if mid.consumer == nil { c.log.Warnf("unable to nack messageID=%+v can not determine topic", msgID) @@ -221,11 +221,11 @@ func (c *multiTopicConsumer) Nack(msg Message) { } func (c *multiTopicConsumer) NackID(msgID MessageID) { - mid := toTrackingMessageID(msgID) - if mid == nil { + if !checkMessageIDType(msgID) { c.log.Warnf("invalid message id type %T", msgID) return } + mid := toTrackingMessageID(msgID) if mid.consumer == nil { c.log.Warnf("unable to nack messageID=%+v can not determine topic", msgID) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 0f7af3de7d..a3dac19c40 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -465,9 +465,6 @@ func (pc *partitionConsumer) ackID(msgID MessageID, withResponse bool) error { } trackingID := toTrackingMessageID(msgID) - if trackingID == nil { - return errors.New("failed to convert trackingMessageID") - } if trackingID != nil && trackingID.ack() { pc.metrics.AcksCounter.Inc() @@ -501,18 +498,34 @@ func (pc *partitionConsumer) sendIndividualAck(msgID MessageID) *ackRequest { } func (pc *partitionConsumer) AckIDWithResponse(msgID MessageID) error { + if !checkMessageIDType(msgID) { + pc.log.Errorf("invalid message id type %T", msgID) + return fmt.Errorf("invalid message id type %T", msgID) + } return pc.ackID(msgID, true) } func (pc *partitionConsumer) AckID(msgID MessageID) error { + if !checkMessageIDType(msgID) { + pc.log.Errorf("invalid message id type %T", msgID) + return fmt.Errorf("invalid message id type %T", msgID) + } return pc.ackID(msgID, false) } func (pc *partitionConsumer) AckIDCumulative(msgID MessageID) error { + if !checkMessageIDType(msgID) { + pc.log.Errorf("invalid message id type %T", msgID) + return fmt.Errorf("invalid message id type %T", msgID) + } return pc.internalAckIDCumulative(msgID, false) } func (pc *partitionConsumer) AckIDWithResponseCumulative(msgID MessageID) error { + if !checkMessageIDType(msgID) { + pc.log.Errorf("invalid message id type %T", msgID) + return fmt.Errorf("invalid message id type %T", msgID) + } return pc.internalAckIDCumulative(msgID, true) } @@ -574,15 +587,17 @@ func (pc *partitionConsumer) sendCumulativeAck(msgID MessageID) *ackRequest { } func (pc *partitionConsumer) NackID(msgID MessageID) { + if !checkMessageIDType(msgID) { + pc.log.Warnf("invalid message id type %T", msgID) + return + } + if cmid, ok := msgID.(*chunkMessageID); ok { pc.unAckChunksTracker.nack(cmid) return } trackingID := toTrackingMessageID(msgID) - if trackingID == nil { - return - } pc.nackTracker.Add(trackingID.messageID) pc.metrics.NacksCounter.Inc() @@ -665,16 +680,20 @@ func (pc *partitionConsumer) Seek(msgID MessageID) error { pc.log.WithField("state", state).Error("Failed to seek by closing or closed consumer") return errors.New("failed to seek by closing or closed consumer") } + + if !checkMessageIDType(msgID) { + pc.log.Errorf("invalid message id type %T", msgID) + return fmt.Errorf("invalid message id type %T", msgID) + } + req := &seekRequest{ doneCh: make(chan struct{}), } if cmid, ok := msgID.(*chunkMessageID); ok { req.msgID = cmid.firstChunkID - } else if tmid := toTrackingMessageID(msgID); tmid != nil { - req.msgID = tmid.messageID } else { - // will never reach - return errors.New("unhandled messageID type") + tmid := toTrackingMessageID(msgID) + req.msgID = tmid.messageID } pc.ackGroupingTracker.flushAndClean() @@ -1812,9 +1831,8 @@ func (c *chunkedMsgCtx) discard(pc *partitionConsumer) { continue } pc.log.Info("Removing chunk message-id", mid.String()) - if tmid := toTrackingMessageID(mid); tmid != nil { - pc.AckID(tmid) - } + tmid := toTrackingMessageID(mid) + pc.AckID(tmid) } } diff --git a/pulsar/consumer_regex.go b/pulsar/consumer_regex.go index fdfecec076..2520af5c8e 100644 --- a/pulsar/consumer_regex.go +++ b/pulsar/consumer_regex.go @@ -174,12 +174,13 @@ func (c *regexConsumer) ReconsumeLaterWithCustomProperties(msg Message, customPr // AckID the consumption of a single message, identified by its MessageID func (c *regexConsumer) AckID(msgID MessageID) error { - mid := toTrackingMessageID(msgID) - if mid == nil { + if !checkMessageIDType(msgID) { c.log.Warnf("invalid message id type %T", msgID) - return errors.New("invalid message id type") + return fmt.Errorf("invalid message id type %T", msgID) } + mid := toTrackingMessageID(msgID) + if mid.consumer == nil { c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID) return errors.New("consumer is nil in consumer_regex") @@ -201,12 +202,13 @@ func (c *regexConsumer) AckCumulative(msg Message) error { // AckIDCumulative the reception of all the messages in the stream up to (and including) // the provided message, identified by its MessageID func (c *regexConsumer) AckIDCumulative(msgID MessageID) error { - mid := toTrackingMessageID(msgID) - if mid == nil { + if !checkMessageIDType(msgID) { c.log.Warnf("invalid message id type %T", msgID) - return errors.New("invalid message id type") + return fmt.Errorf("invalid message id type %T", msgID) } + mid := toTrackingMessageID(msgID) + if mid.consumer == nil { c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID) return errors.New("unable to ack message because consumer is nil") @@ -222,11 +224,11 @@ func (c *regexConsumer) AckIDCumulative(msgID MessageID) error { func (c *regexConsumer) Nack(msg Message) { if c.options.EnableDefaultNackBackoffPolicy || c.options.NackBackoffPolicy != nil { msgID := msg.ID() - mid := toTrackingMessageID(msgID) - if mid == nil { + if !checkMessageIDType(msgID) { c.log.Warnf("invalid message id type %T", msgID) return } + mid := toTrackingMessageID(msgID) if mid.consumer == nil { c.log.Warnf("unable to nack messageID=%+v can not determine topic", msgID) @@ -240,12 +242,13 @@ func (c *regexConsumer) Nack(msg Message) { } func (c *regexConsumer) NackID(msgID MessageID) { - mid := toTrackingMessageID(msgID) - if mid == nil { + if !checkMessageIDType(msgID) { c.log.Warnf("invalid message id type %T", msgID) return } + mid := toTrackingMessageID(msgID) + if mid.consumer == nil { c.log.Warnf("unable to nack messageID=%+v can not determine topic", msgID) return diff --git a/pulsar/impl_message.go b/pulsar/impl_message.go index 68ddecd449..9c56070295 100644 --- a/pulsar/impl_message.go +++ b/pulsar/impl_message.go @@ -213,6 +213,16 @@ func newMessageID(ledgerID int64, entryID int64, batchIdx int32, partitionIdx in } } +func fromMessageID(msgID MessageID) *messageID { + return &messageID{ + ledgerID: msgID.LedgerID(), + entryID: msgID.EntryID(), + batchIdx: msgID.BatchIdx(), + partitionIdx: msgID.PartitionIdx(), + batchSize: msgID.BatchSize(), + } +} + func newTrackingMessageID(ledgerID int64, entryID int64, batchIdx int32, partitionIdx int32, batchSize int32, tracker *ackTracker) *trackingMessageID { return &trackingMessageID{ @@ -228,22 +238,26 @@ func newTrackingMessageID(ledgerID int64, entryID int64, batchIdx int32, partiti } } -func toTrackingMessageID(msgID MessageID) *trackingMessageID { - if mid, ok := msgID.(*messageID); ok { - return &trackingMessageID{ - messageID: mid, - receivedTime: time.Now(), - } - } else if mid, ok := msgID.(*trackingMessageID); ok { +// checkMessageIDType checks if the MessageID is user-defined +func checkMessageIDType(msgID MessageID) (valid bool) { + switch msgID.(type) { + case *trackingMessageID: + return true + case *chunkMessageID: + return true + case *messageID: + return true + default: + return false + } +} + +func toTrackingMessageID(msgID MessageID) (trackingMsgID *trackingMessageID) { + if mid, ok := msgID.(*trackingMessageID); ok { return mid - } else if cmid, ok := msgID.(*chunkMessageID); ok { - return &trackingMessageID{ - messageID: cmid.messageID, - receivedTime: cmid.receivedTime, - consumer: cmid.consumer, - } - } else { - return nil + } + return &trackingMessageID{ + messageID: fromMessageID(msgID), } } diff --git a/pulsar/reader_impl.go b/pulsar/reader_impl.go index 68dd0843ee..c7620ad028 100644 --- a/pulsar/reader_impl.go +++ b/pulsar/reader_impl.go @@ -51,8 +51,8 @@ func newReader(client *client, options ReaderOptions) (Reader, error) { return nil, newError(InvalidConfiguration, "StartMessageID is required") } - startMessageID := toTrackingMessageID(options.StartMessageID) - if startMessageID == nil { + var startMessageID *trackingMessageID + if !checkMessageIDType(options.StartMessageID) { // a custom type satisfying MessageID may not be a messageID or trackingMessageID // so re-create messageID using its data deserMsgID, err := deserializeMessageID(options.StartMessageID.Serialize()) @@ -60,10 +60,9 @@ func newReader(client *client, options ReaderOptions) (Reader, error) { return nil, err } // de-serialized MessageID is a messageID - startMessageID = &trackingMessageID{ - messageID: deserMsgID.(*messageID), - receivedTime: time.Now(), - } + startMessageID = toTrackingMessageID(deserMsgID) + } else { + startMessageID = toTrackingMessageID(options.StartMessageID) } subscriptionName := options.SubscriptionName @@ -148,12 +147,10 @@ func (r *reader) Next(ctx context.Context) (Message, error) { // Acknowledge message immediately because the reader is based on non-durable subscription. When it reconnects, // it will specify the subscription position anyway msgID := cm.Message.ID() - if mid := toTrackingMessageID(msgID); mid != nil { - r.pc.lastDequeuedMsg = mid - r.pc.AckID(mid) - return cm.Message, nil - } - return nil, newError(InvalidMessage, fmt.Sprintf("invalid message id type %T", msgID)) + mid := toTrackingMessageID(msgID) + r.pc.lastDequeuedMsg = mid + r.pc.AckID(mid) + return cm.Message, nil case <-ctx.Done(): return nil, ctx.Err() } @@ -202,10 +199,6 @@ func (r *reader) Close() { func (r *reader) messageID(msgID MessageID) *trackingMessageID { mid := toTrackingMessageID(msgID) - if mid == nil { - r.log.Warnf("invalid message id type %T", msgID) - return nil - } partition := int(mid.partitionIdx) // did we receive a valid partition index? @@ -221,6 +214,11 @@ func (r *reader) Seek(msgID MessageID) error { r.Lock() defer r.Unlock() + if !checkMessageIDType(msgID) { + r.log.Warnf("invalid message id type %T", msgID) + return fmt.Errorf("invalid message id type %T", msgID) + } + mid := r.messageID(msgID) if mid == nil { return nil From 07f97d9a8c32031ab18b8f8afa82a06f7804a634 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Tue, 7 Mar 2023 20:06:25 +0800 Subject: [PATCH 029/348] [fix] Setup rate limiter for TestChunksEnqueueFailed to reduce flaky (#982) --- pulsar/message_chunking_test.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pulsar/message_chunking_test.go b/pulsar/message_chunking_test.go index aac87c7c13..0e1214d311 100644 --- a/pulsar/message_chunking_test.go +++ b/pulsar/message_chunking_test.go @@ -22,6 +22,7 @@ import ( "errors" "fmt" "math/rand" + "net/http" "strings" "sync" "testing" @@ -291,6 +292,13 @@ func TestChunksEnqueueFailed(t *testing.T) { assert.NotNil(t, producer) defer producer.Close() + // Reduce publish rate to prevent the producer sending messages too fast + url := adminURL + "/" + "admin/v2/persistent/public/default/" + topic + "/publishRate" + makeHTTPCall(t, http.MethodPost, url, "{\"publishThrottlingRateInMsg\": 1,\"publishThrottlingRateInByte\": 1000}") + + // Need to wait some time to let the rate limiter take effect + time.Sleep(2 * time.Second) + ID, err := producer.Send(context.Background(), &ProducerMessage{ Payload: createTestMessagePayload(1000), }) From 69855216709c9480b3578d6bb57fee73867b27d2 Mon Sep 17 00:00:00 2001 From: Sekfung Lau Date: Wed, 8 Mar 2023 18:38:00 +0800 Subject: [PATCH 030/348] fix: prevent rpc client panic on rpc response if `ProducerReady` is nil (#973) ### Motivation prevent rpc client panic on rpc response if `ProducerReady` is nil ### Modifications use `ProducerSuccess.GetProducerReady` instead of `ProducerSuccess.ProducerReady` --- pulsar/internal/connection.go | 2 +- pulsar/internal/rpc_client.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 55d04f3e82..59aad1675e 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -525,7 +525,7 @@ func (c *connection) internalReceivedCommand(cmd *pb.BaseCommand, headersAndPayl c.handleResponse(cmd.Success.GetRequestId(), cmd) case pb.BaseCommand_PRODUCER_SUCCESS: - if !*cmd.ProducerSuccess.ProducerReady { + if !cmd.ProducerSuccess.GetProducerReady() { request, ok := c.findPendingRequest(cmd.ProducerSuccess.GetRequestId()) if ok { request.callback(cmd, nil) diff --git a/pulsar/internal/rpc_client.go b/pulsar/internal/rpc_client.go index 0ee8ca93c7..2213083d85 100644 --- a/pulsar/internal/rpc_client.go +++ b/pulsar/internal/rpc_client.go @@ -138,7 +138,7 @@ func (c *rpcClient) Request(logicalAddr *url.URL, physicalAddr *url.URL, request // Ignoring producer not ready response. // Continue to wait for the producer to create successfully if res.error == nil && *res.RPCResult.Response.Type == pb.BaseCommand_PRODUCER_SUCCESS { - if !*res.RPCResult.Response.ProducerSuccess.ProducerReady { + if !res.RPCResult.Response.ProducerSuccess.GetProducerReady() { timeoutCh = nil break } From 5277f3fd51a6795276a9e39fd65164315b250b85 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Wed, 8 Mar 2023 23:06:02 +0800 Subject: [PATCH 031/348] Fix ack grouping tracker is duplicate method will be blocked. (#986) --- pulsar/ack_grouping_tracker.go | 191 ++++++++++++---------------- pulsar/ack_grouping_tracker_test.go | 12 ++ 2 files changed, 95 insertions(+), 108 deletions(-) diff --git a/pulsar/ack_grouping_tracker.go b/pulsar/ack_grouping_tracker.go index dbc70f5d70..22aba115e2 100644 --- a/pulsar/ack_grouping_tracker.go +++ b/pulsar/ack_grouping_tracker.go @@ -18,6 +18,7 @@ package pulsar import ( + "sync" "time" "github.com/bits-and-blooms/bitset" @@ -37,14 +38,6 @@ type ackGroupingTracker interface { close() } -type ackFlushType int - -const ( - flushOnly ackFlushType = iota - flushAndClean - flushAndClose -) - func newAckGroupingTracker(options *AckGroupingOptions, ackIndividual func(id MessageID), ackCumulative func(id MessageID)) ackGroupingTracker { @@ -62,7 +55,7 @@ func newAckGroupingTracker(options *AckGroupingOptions, } } - c := &cachedAcks{ + t := &timedAckGroupingTracker{ singleAcks: make([]MessageID, options.MaxSize), pendingAcks: make(map[int64]*bitset.BitSet), lastCumulativeAck: EarliestMessageID(), @@ -74,51 +67,24 @@ func newAckGroupingTracker(options *AckGroupingOptions, ackIndividual(id) } }, + options: *options, + tick: time.NewTicker(time.Hour), + donCh: make(chan struct{}), } - timeout := time.NewTicker(time.Hour) if options.MaxTime > 0 { - timeout = time.NewTicker(options.MaxTime) + t.tick = time.NewTicker(options.MaxTime) } else { - timeout.Stop() - } - t := &timedAckGroupingTracker{ - ackIndividualCh: make(chan MessageID), - ackCumulativeCh: make(chan MessageID), - duplicateIDCh: make(chan MessageID), - duplicateResultCh: make(chan bool), - flushCh: make(chan ackFlushType), - waitFlushCh: make(chan bool), + t.tick.Stop() } + go func() { for { select { - case id := <-t.ackIndividualCh: - if c.addAndCheckIfFull(id) { - c.flushIndividualAcks() - if options.MaxTime > 0 { - timeout.Reset(options.MaxTime) - } - } - case id := <-t.ackCumulativeCh: - c.tryUpdateLastCumulativeAck(id) - if options.MaxTime <= 0 { - c.flushCumulativeAck() - } - case id := <-t.duplicateIDCh: - t.duplicateResultCh <- c.isDuplicate(id) - case <-timeout.C: - c.flush() - case ackFlushType := <-t.flushCh: - timeout.Stop() - c.flush() - if ackFlushType == flushAndClean { - c.clean() - } - t.waitFlushCh <- true - if ackFlushType == flushAndClose { - return - } + case <-t.donCh: + return + case <-t.tick.C: + t.flush() } } }() @@ -151,27 +117,9 @@ func (i *immediateAckGroupingTracker) flushAndClean() { func (i *immediateAckGroupingTracker) close() { } -type cachedAcks struct { - singleAcks []MessageID - index int - - // Key is the hash code of the ledger id and the netry id, - // Value is the bit set that represents which messages are acknowledged if the entry stores a batch. - // The bit 1 represents the message has been acknowledged, i.e. the bits "111" represents all messages - // in the batch whose batch size is 3 are not acknowledged. - // After the 1st message (i.e. batch index is 0) is acknowledged, the bits will become "011". - // Value is nil if the entry represents a single message. - pendingAcks map[int64]*bitset.BitSet - - lastCumulativeAck MessageID - cumulativeAckRequired bool - - ackIndividual func(id MessageID) - ackCumulative func(id MessageID) - ackList func(ids []MessageID) -} - -func (t *cachedAcks) addAndCheckIfFull(id MessageID) bool { +func (t *timedAckGroupingTracker) addAndCheckIfFull(id MessageID) bool { + t.mutex.Lock() + defer t.mutex.Unlock() t.singleAcks[t.index] = id t.index++ key := messageIDHash(id) @@ -193,30 +141,18 @@ func (t *cachedAcks) addAndCheckIfFull(id MessageID) bool { return t.index == len(t.singleAcks) } -func (t *cachedAcks) tryUpdateLastCumulativeAck(id MessageID) { +func (t *timedAckGroupingTracker) tryUpdateLastCumulativeAck(id MessageID) { + t.mutex.Lock() + defer t.mutex.Unlock() if messageIDCompare(t.lastCumulativeAck, id) < 0 { t.lastCumulativeAck = id t.cumulativeAckRequired = true } } -func (t *cachedAcks) isDuplicate(id MessageID) bool { - if messageIDCompare(t.lastCumulativeAck, id) >= 0 { - return true - } - ackSet, found := t.pendingAcks[messageIDHash(id)] - if !found { - return false - } - if ackSet == nil || !messageIDIsBatch(id) { - // NOTE: should we panic when ackSet != nil and messageIDIsBatch(id) is true? - return true - } - // 0 represents the message has been acknowledged - return !ackSet.Test(uint(id.BatchIdx())) -} - -func (t *cachedAcks) flushIndividualAcks() { +func (t *timedAckGroupingTracker) flushIndividualAcks() { + t.mutex.Lock() + defer t.mutex.Unlock() if t.index > 0 { t.ackList(t.singleAcks[0:t.index]) for _, id := range t.singleAcks[0:t.index] { @@ -239,19 +175,18 @@ func (t *cachedAcks) flushIndividualAcks() { } } -func (t *cachedAcks) flushCumulativeAck() { +func (t *timedAckGroupingTracker) flushCumulativeAck() { + t.mutex.Lock() + defer t.mutex.Unlock() if t.cumulativeAckRequired { t.ackCumulative(t.lastCumulativeAck) t.cumulativeAckRequired = false } } -func (t *cachedAcks) flush() { - t.flushIndividualAcks() - t.flushCumulativeAck() -} - -func (t *cachedAcks) clean() { +func (t *timedAckGroupingTracker) clean() { + t.mutex.Lock() + defer t.mutex.Unlock() maxSize := len(t.singleAcks) t.singleAcks = make([]MessageID, maxSize) t.index = 0 @@ -261,38 +196,78 @@ func (t *cachedAcks) clean() { } type timedAckGroupingTracker struct { - ackIndividualCh chan MessageID - ackCumulativeCh chan MessageID - duplicateIDCh chan MessageID - duplicateResultCh chan bool - flushCh chan ackFlushType - waitFlushCh chan bool + singleAcks []MessageID + index int + + // Key is the hash code of the ledger id and the netry id, + // Value is the bit set that represents which messages are acknowledged if the entry stores a batch. + // The bit 1 represents the message has been acknowledged, i.e. the bits "111" represents all messages + // in the batch whose batch size is 3 are not acknowledged. + // After the 1st message (i.e. batch index is 0) is acknowledged, the bits will become "011". + // Value is nil if the entry represents a single message. + pendingAcks map[int64]*bitset.BitSet + + lastCumulativeAck MessageID + cumulativeAckRequired bool + + ackIndividual func(id MessageID) + ackCumulative func(id MessageID) + ackList func(ids []MessageID) + + options AckGroupingOptions + donCh chan struct{} + tick *time.Ticker + + mutex sync.RWMutex } func (t *timedAckGroupingTracker) add(id MessageID) { - t.ackIndividualCh <- id + if t.addAndCheckIfFull(id) { + t.flushIndividualAcks() + if t.options.MaxTime > 0 { + t.tick.Reset(t.options.MaxTime) + } + } } func (t *timedAckGroupingTracker) addCumulative(id MessageID) { - t.ackCumulativeCh <- id + t.tryUpdateLastCumulativeAck(id) + if t.options.MaxTime <= 0 { + t.flushCumulativeAck() + } } func (t *timedAckGroupingTracker) isDuplicate(id MessageID) bool { - t.duplicateIDCh <- id - return <-t.duplicateResultCh + t.mutex.RLock() + if messageIDCompare(t.lastCumulativeAck, id) >= 0 { + t.mutex.RUnlock() + return true + } + ackSet, found := t.pendingAcks[messageIDHash(id)] + if !found { + t.mutex.RUnlock() + return false + } + t.mutex.RUnlock() + if ackSet == nil || !messageIDIsBatch(id) { + // NOTE: should we panic when ackSet != nil and messageIDIsBatch(id) is true? + return true + } + // 0 represents the message has been acknowledged + return !ackSet.Test(uint(id.BatchIdx())) } func (t *timedAckGroupingTracker) flush() { - t.flushCh <- flushOnly - <-t.waitFlushCh + t.flushIndividualAcks() + t.flushCumulativeAck() } func (t *timedAckGroupingTracker) flushAndClean() { - t.flushCh <- flushAndClean - <-t.waitFlushCh + t.flush() + t.clean() } func (t *timedAckGroupingTracker) close() { - t.flushCh <- flushAndClose - <-t.waitFlushCh + t.flushAndClean() + close(t.donCh) } diff --git a/pulsar/ack_grouping_tracker_test.go b/pulsar/ack_grouping_tracker_test.go index e7a67258d6..41d24d4d78 100644 --- a/pulsar/ack_grouping_tracker_test.go +++ b/pulsar/ack_grouping_tracker_test.go @@ -195,3 +195,15 @@ func TestTimedTrackerIsDuplicate(t *testing.T) { assert.False(t, tracker.isDuplicate(&messageID{batchIdx: 1, batchSize: 3})) assert.False(t, tracker.isDuplicate(&messageID{batchIdx: 2, batchSize: 3})) } + +func TestDuplicateAfterClose(t *testing.T) { + var acker mockAcker + tracker := newAckGroupingTracker(&AckGroupingOptions{MaxSize: 3, MaxTime: 0}, + func(id MessageID) { acker.ack(id) }, func(id MessageID) { acker.ackCumulative(id) }) + + tracker.add(&messageID{ledgerID: 1}) + assert.True(t, tracker.isDuplicate(&messageID{ledgerID: 1})) + + tracker.close() + assert.False(t, tracker.isDuplicate(&messageID{ledgerID: 1})) +} From e269c42887c09c35d3f2cdcb4bc5a1699ccf2711 Mon Sep 17 00:00:00 2001 From: CrazyCollin Date: Thu, 9 Mar 2023 21:21:25 +0800 Subject: [PATCH 032/348] [feat] Expose the chunk config of consumer to the reader (#987) * [feat] Expose the chunk config of consumer to the reader * add test for reader's chunk config * refactoring some code * Update pulsar/reader.go Co-authored-by: Zike Yang --------- Co-authored-by: Zike Yang --- pulsar/reader.go | 10 ++++++++++ pulsar/reader_impl.go | 41 ++++++++++++++++++++++++++--------------- pulsar/reader_test.go | 39 +++++++++++++++++++++++++++++++++++++++ 3 files changed, 75 insertions(+), 15 deletions(-) diff --git a/pulsar/reader.go b/pulsar/reader.go index e4679abee2..d58d06f6fe 100644 --- a/pulsar/reader.go +++ b/pulsar/reader.go @@ -92,6 +92,16 @@ type ReaderOptions struct { // BackoffPolicy parameterize the following options in the reconnection logic to // allow users to customize the reconnection logic (minBackoff, maxBackoff and jitterPercentage) BackoffPolicy internal.BackoffPolicy + + // MaxPendingChunkedMessage sets the maximum pending chunked messages. (default: 100) + MaxPendingChunkedMessage int + + // ExpireTimeOfIncompleteChunk sets the expiry time of discarding incomplete chunked message. (default: 60 seconds) + ExpireTimeOfIncompleteChunk time.Duration + + // AutoAckIncompleteChunk sets whether reader auto acknowledges incomplete chunked message when it should + // be removed (e.g.the chunked message pending queue is full). (default: false) + AutoAckIncompleteChunk bool } // Reader can be used to scan through all the messages currently available in a topic. diff --git a/pulsar/reader_impl.go b/pulsar/reader_impl.go index c7620ad028..36b492abea 100644 --- a/pulsar/reader_impl.go +++ b/pulsar/reader_impl.go @@ -90,22 +90,33 @@ func newReader(client *client, options ReaderOptions) (Reader, error) { options.Decryption.MessageCrypto = messageCrypto } + if options.MaxPendingChunkedMessage == 0 { + options.MaxPendingChunkedMessage = 100 + } + + if options.ExpireTimeOfIncompleteChunk == 0 { + options.ExpireTimeOfIncompleteChunk = time.Minute + } + consumerOptions := &partitionConsumerOpts{ - topic: options.Topic, - consumerName: options.Name, - subscription: subscriptionName, - subscriptionType: Exclusive, - receiverQueueSize: receiverQueueSize, - startMessageID: startMessageID, - startMessageIDInclusive: options.StartMessageIDInclusive, - subscriptionMode: nonDurable, - readCompacted: options.ReadCompacted, - metadata: options.Properties, - nackRedeliveryDelay: defaultNackRedeliveryDelay, - replicateSubscriptionState: false, - decryption: options.Decryption, - schema: options.Schema, - backoffPolicy: options.BackoffPolicy, + topic: options.Topic, + consumerName: options.Name, + subscription: subscriptionName, + subscriptionType: Exclusive, + receiverQueueSize: receiverQueueSize, + startMessageID: startMessageID, + startMessageIDInclusive: options.StartMessageIDInclusive, + subscriptionMode: nonDurable, + readCompacted: options.ReadCompacted, + metadata: options.Properties, + nackRedeliveryDelay: defaultNackRedeliveryDelay, + replicateSubscriptionState: false, + decryption: options.Decryption, + schema: options.Schema, + backoffPolicy: options.BackoffPolicy, + maxPendingChunkedMessage: options.MaxPendingChunkedMessage, + expireTimeOfIncompleteChunk: options.ExpireTimeOfIncompleteChunk, + autoAckIncompleteChunk: options.AutoAckIncompleteChunk, } reader := &reader{ diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index 35431874bc..0a1b2a1df4 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -70,6 +70,45 @@ func TestReaderConfigSubscribeName(t *testing.T) { assert.NotNil(t, consumer) } +func TestReaderConfigChunk(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + if err != nil { + t.Fatal(err) + } + defer client.Close() + + r1, err := client.CreateReader(ReaderOptions{ + Topic: "my-topic1", + StartMessageID: EarliestMessageID(), + MaxPendingChunkedMessage: 50, + ExpireTimeOfIncompleteChunk: 30 * time.Second, + AutoAckIncompleteChunk: true, + }) + assert.Nil(t, err) + defer r1.Close() + + // verify specified chunk options + pcOpts := r1.(*reader).pc.options + assert.Equal(t, 50, pcOpts.maxPendingChunkedMessage) + assert.Equal(t, 30*time.Second, pcOpts.expireTimeOfIncompleteChunk) + assert.True(t, pcOpts.autoAckIncompleteChunk) + + r2, err := client.CreateReader(ReaderOptions{ + Topic: "my-topic2", + StartMessageID: EarliestMessageID(), + }) + assert.Nil(t, err) + defer r2.Close() + + // verify default chunk options + pcOpts = r2.(*reader).pc.options + assert.Equal(t, 100, pcOpts.maxPendingChunkedMessage) + assert.Equal(t, time.Minute, pcOpts.expireTimeOfIncompleteChunk) + assert.False(t, pcOpts.autoAckIncompleteChunk) +} + func TestReader(t *testing.T) { client, err := NewClient(ClientOptions{ URL: lookupURL, From 352c463194916cab7772c0729c9cdac14404e8bd Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 10 Mar 2023 09:57:38 +0800 Subject: [PATCH 033/348] Optimize batch index ACK performance (#988) ### Motivation Currently, when `EnableBatchIndexAck` is true, the ACK performance is very poor. There are two main reasons: 1. Acknowledgment by list is not supported. It means that even N MessageIDs are grouped, there are still N ACK requests to send. 2. The implementation of ACK grouping tracker is wrong. Give a batch that has N messages, when batch index ACK is enabled, each MessageID is cached. However, after all these N MessageIDs arrived, the current implementation does not clear them. ### Modifications - Add a `func(id []*pb.MessageIdData)` to the ACK grouping tracker. When flushing individual ACKs, construct the slice and wrap the slice to `CommandAck` directly. - Refactor the implementation of the ACK grouping tracker: - Do not save each MessageID instance, instead, save the ledger id and the entry id as the key of `pendingAcks`. - Release the mutex before calling ACK functions - Add `TestTrackerPendingAcks` to verify the list of MessageIDs to ACK. After this change, the ACK order cannot be guaranteed, sort the acknowledged MessageIDs in the `ack_grouping_tracker_test.go`. --- pulsar/ack_grouping_tracker.go | 281 ++++++++++++++-------------- pulsar/ack_grouping_tracker_test.go | 54 +++++- pulsar/consumer_partition.go | 13 +- pulsar/consumer_partition_test.go | 6 +- pulsar/message.go | 8 - 5 files changed, 196 insertions(+), 166 deletions(-) diff --git a/pulsar/ack_grouping_tracker.go b/pulsar/ack_grouping_tracker.go index 22aba115e2..dd051868d7 100644 --- a/pulsar/ack_grouping_tracker.go +++ b/pulsar/ack_grouping_tracker.go @@ -19,8 +19,10 @@ package pulsar import ( "sync" + "sync/atomic" "time" + pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" "github.com/bits-and-blooms/bitset" ) @@ -40,7 +42,8 @@ type ackGroupingTracker interface { func newAckGroupingTracker(options *AckGroupingOptions, ackIndividual func(id MessageID), - ackCumulative func(id MessageID)) ackGroupingTracker { + ackCumulative func(id MessageID), + ackList func(ids []*pb.MessageIdData)) ackGroupingTracker { if options == nil { options = &AckGroupingOptions{ MaxSize: 1000, @@ -56,38 +59,28 @@ func newAckGroupingTracker(options *AckGroupingOptions, } t := &timedAckGroupingTracker{ - singleAcks: make([]MessageID, options.MaxSize), - pendingAcks: make(map[int64]*bitset.BitSet), - lastCumulativeAck: EarliestMessageID(), - ackIndividual: ackIndividual, + maxNumAcks: int(options.MaxSize), ackCumulative: ackCumulative, - ackList: func(ids []MessageID) { - // TODO: support ack a list of MessageIDs - for _, id := range ids { - ackIndividual(id) - } - }, - options: *options, - tick: time.NewTicker(time.Hour), - donCh: make(chan struct{}), + ackList: ackList, + pendingAcks: make(map[[2]uint64]*bitset.BitSet), + lastCumulativeAck: EarliestMessageID(), } if options.MaxTime > 0 { - t.tick = time.NewTicker(options.MaxTime) - } else { - t.tick.Stop() + t.ticker = time.NewTicker(options.MaxTime) + t.exitCh = make(chan struct{}) + go func() { + for { + select { + case <-t.exitCh: + return + case <-t.ticker.C: + t.flush() + } + } + }() } - go func() { - for { - select { - case <-t.donCh: - return - case <-t.tick.C: - t.flush() - } - } - }() return t } @@ -117,157 +110,157 @@ func (i *immediateAckGroupingTracker) flushAndClean() { func (i *immediateAckGroupingTracker) close() { } -func (t *timedAckGroupingTracker) addAndCheckIfFull(id MessageID) bool { - t.mutex.Lock() - defer t.mutex.Unlock() - t.singleAcks[t.index] = id - t.index++ - key := messageIDHash(id) - ackSet, found := t.pendingAcks[key] - if !found { - if messageIDIsBatch(id) { - ackSet = bitset.New(uint(id.BatchSize())) - for i := 0; i < int(id.BatchSize()); i++ { - ackSet.Set(uint(i)) - } - t.pendingAcks[key] = ackSet - } else { - t.pendingAcks[key] = nil - } - } - if ackSet != nil { - ackSet.Clear(uint(id.BatchIdx())) - } - return t.index == len(t.singleAcks) -} - -func (t *timedAckGroupingTracker) tryUpdateLastCumulativeAck(id MessageID) { - t.mutex.Lock() - defer t.mutex.Unlock() - if messageIDCompare(t.lastCumulativeAck, id) < 0 { - t.lastCumulativeAck = id - t.cumulativeAckRequired = true - } -} - -func (t *timedAckGroupingTracker) flushIndividualAcks() { - t.mutex.Lock() - defer t.mutex.Unlock() - if t.index > 0 { - t.ackList(t.singleAcks[0:t.index]) - for _, id := range t.singleAcks[0:t.index] { - key := messageIDHash(id) - ackSet, found := t.pendingAcks[key] - if !found { - continue - } - if ackSet == nil { - delete(t.pendingAcks, key) - } else { - ackSet.Clear(uint(id.BatchIdx())) - if ackSet.None() { // all messages have been acknowledged - delete(t.pendingAcks, key) - } - } - delete(t.pendingAcks, messageIDHash(id)) - } - t.index = 0 - } -} - -func (t *timedAckGroupingTracker) flushCumulativeAck() { - t.mutex.Lock() - defer t.mutex.Unlock() - if t.cumulativeAckRequired { - t.ackCumulative(t.lastCumulativeAck) - t.cumulativeAckRequired = false - } -} - -func (t *timedAckGroupingTracker) clean() { - t.mutex.Lock() - defer t.mutex.Unlock() - maxSize := len(t.singleAcks) - t.singleAcks = make([]MessageID, maxSize) - t.index = 0 - t.pendingAcks = make(map[int64]*bitset.BitSet) - t.lastCumulativeAck = EarliestMessageID() - t.cumulativeAckRequired = false -} - type timedAckGroupingTracker struct { - singleAcks []MessageID - index int + sync.RWMutex - // Key is the hash code of the ledger id and the netry id, + maxNumAcks int + ackCumulative func(id MessageID) + ackList func(ids []*pb.MessageIdData) + ticker *time.Ticker + + // Key is the pair of the ledger id and the entry id, // Value is the bit set that represents which messages are acknowledged if the entry stores a batch. // The bit 1 represents the message has been acknowledged, i.e. the bits "111" represents all messages // in the batch whose batch size is 3 are not acknowledged. // After the 1st message (i.e. batch index is 0) is acknowledged, the bits will become "011". // Value is nil if the entry represents a single message. - pendingAcks map[int64]*bitset.BitSet + pendingAcks map[[2]uint64]*bitset.BitSet lastCumulativeAck MessageID - cumulativeAckRequired bool - - ackIndividual func(id MessageID) - ackCumulative func(id MessageID) - ackList func(ids []MessageID) - - options AckGroupingOptions - donCh chan struct{} - tick *time.Ticker + cumulativeAckRequired int32 - mutex sync.RWMutex + exitCh chan struct{} } func (t *timedAckGroupingTracker) add(id MessageID) { - if t.addAndCheckIfFull(id) { - t.flushIndividualAcks() - if t.options.MaxTime > 0 { - t.tick.Reset(t.options.MaxTime) + if acks := t.tryAddIndividual(id); acks != nil { + t.flushIndividual(acks) + } +} + +func (t *timedAckGroupingTracker) tryAddIndividual(id MessageID) map[[2]uint64]*bitset.BitSet { + t.Lock() + defer t.Unlock() + key := [2]uint64{uint64(id.LedgerID()), uint64(id.EntryID())} + + batchIdx := id.BatchIdx() + batchSize := id.BatchSize() + + if batchIdx >= 0 && batchSize > 0 { + bs, found := t.pendingAcks[key] + if !found { + if batchSize > 1 { + bs = bitset.New(uint(batchSize)) + for i := uint(0); i < uint(batchSize); i++ { + bs.Set(i) + } + } + t.pendingAcks[key] = bs } + if bs != nil { + bs.Clear(uint(batchIdx)) + } + } else { + t.pendingAcks[key] = nil } + + if len(t.pendingAcks) >= t.maxNumAcks { + pendingAcks := t.pendingAcks + t.pendingAcks = make(map[[2]uint64]*bitset.BitSet) + return pendingAcks + } + return nil } func (t *timedAckGroupingTracker) addCumulative(id MessageID) { - t.tryUpdateLastCumulativeAck(id) - if t.options.MaxTime <= 0 { - t.flushCumulativeAck() + if t.tryUpdateCumulative(id) && t.ticker == nil { + t.ackCumulative(id) } } +func (t *timedAckGroupingTracker) tryUpdateCumulative(id MessageID) bool { + t.Lock() + defer t.Unlock() + if messageIDCompare(t.lastCumulativeAck, id) < 0 { + t.lastCumulativeAck = id + atomic.StoreInt32(&t.cumulativeAckRequired, 1) + return true + } + return false +} + func (t *timedAckGroupingTracker) isDuplicate(id MessageID) bool { - t.mutex.RLock() + t.RLock() + defer t.RUnlock() if messageIDCompare(t.lastCumulativeAck, id) >= 0 { - t.mutex.RUnlock() return true } - ackSet, found := t.pendingAcks[messageIDHash(id)] - if !found { - t.mutex.RUnlock() - return false - } - t.mutex.RUnlock() - if ackSet == nil || !messageIDIsBatch(id) { - // NOTE: should we panic when ackSet != nil and messageIDIsBatch(id) is true? - return true + key := [2]uint64{uint64(id.LedgerID()), uint64(id.EntryID())} + if bs, found := t.pendingAcks[key]; found { + if bs == nil { + return true + } + if !bs.Test(uint(id.BatchIdx())) { + return true + } } - // 0 represents the message has been acknowledged - return !ackSet.Test(uint(id.BatchIdx())) + return false } func (t *timedAckGroupingTracker) flush() { - t.flushIndividualAcks() - t.flushCumulativeAck() + if acks := t.clearPendingAcks(); len(acks) > 0 { + t.flushIndividual(acks) + } + if atomic.CompareAndSwapInt32(&t.cumulativeAckRequired, 1, 0) { + t.RLock() + id := t.lastCumulativeAck + t.RUnlock() + t.ackCumulative(id) + } } func (t *timedAckGroupingTracker) flushAndClean() { - t.flush() - t.clean() + if acks := t.clearPendingAcks(); len(acks) > 0 { + t.flushIndividual(acks) + } + if atomic.CompareAndSwapInt32(&t.cumulativeAckRequired, 1, 0) { + t.Lock() + id := t.lastCumulativeAck + t.lastCumulativeAck = EarliestMessageID() + t.Unlock() + t.ackCumulative(id) + } +} + +func (t *timedAckGroupingTracker) clearPendingAcks() map[[2]uint64]*bitset.BitSet { + t.Lock() + defer t.Unlock() + pendingAcks := t.pendingAcks + t.pendingAcks = make(map[[2]uint64]*bitset.BitSet) + return pendingAcks } func (t *timedAckGroupingTracker) close() { t.flushAndClean() - close(t.donCh) + if t.exitCh != nil { + close(t.exitCh) + } +} + +func (t *timedAckGroupingTracker) flushIndividual(pendingAcks map[[2]uint64]*bitset.BitSet) { + msgIDs := make([]*pb.MessageIdData, 0, len(pendingAcks)) + for k, v := range pendingAcks { + ledgerID := k[0] + entryID := k[1] + msgID := &pb.MessageIdData{LedgerId: &ledgerID, EntryId: &entryID} + if v != nil && !v.None() { + bytes := v.Bytes() + msgID.AckSet = make([]int64, len(bytes)) + for i := 0; i < len(bytes); i++ { + msgID.AckSet[i] = int64(bytes[i]) + } + } + msgIDs = append(msgIDs, msgID) + } + t.ackList(msgIDs) } diff --git a/pulsar/ack_grouping_tracker_test.go b/pulsar/ack_grouping_tracker_test.go index 41d24d4d78..0a794f6403 100644 --- a/pulsar/ack_grouping_tracker_test.go +++ b/pulsar/ack_grouping_tracker_test.go @@ -19,11 +19,13 @@ package pulsar import ( "fmt" + "sort" "sync" "sync/atomic" "testing" "time" + pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" "github.com/stretchr/testify/assert" ) @@ -45,7 +47,8 @@ func TestNoCacheTracker(t *testing.T) { ledgerID1 := int64(-1) tracker := newAckGroupingTracker(&option, func(id MessageID) { ledgerID0 = id.LedgerID() }, - func(id MessageID) { ledgerID1 = id.LedgerID() }) + func(id MessageID) { ledgerID1 = id.LedgerID() }, + nil) tracker.add(&messageID{ledgerID: 1}) assert.Equal(t, atomic.LoadInt64(&ledgerID0), int64(1)) @@ -61,10 +64,12 @@ type mockAcker struct { cumulativeLedgerID int64 } -func (a *mockAcker) ack(id MessageID) { +func (a *mockAcker) ack(ids []*pb.MessageIdData) { defer a.Unlock() a.Lock() - a.ledgerIDs = append(a.ledgerIDs, id.LedgerID()) + for _, id := range ids { + a.ledgerIDs = append(a.ledgerIDs, int64(*id.LedgerId)) + } } func (a *mockAcker) ackCumulative(id MessageID) { @@ -74,6 +79,8 @@ func (a *mockAcker) ackCumulative(id MessageID) { func (a *mockAcker) getLedgerIDs() []int64 { defer a.Unlock() a.Lock() + + sort.Slice(a.ledgerIDs, func(i, j int) bool { return a.ledgerIDs[i] < a.ledgerIDs[j] }) return a.ledgerIDs } @@ -88,8 +95,8 @@ func (a *mockAcker) reset() { func TestCachedTracker(t *testing.T) { var acker mockAcker - tracker := newAckGroupingTracker(&AckGroupingOptions{MaxSize: 3, MaxTime: 0}, - func(id MessageID) { acker.ack(id) }, func(id MessageID) { acker.ackCumulative(id) }) + tracker := newAckGroupingTracker(&AckGroupingOptions{MaxSize: 3, MaxTime: 0}, nil, + func(id MessageID) { acker.ackCumulative(id) }, func(ids []*pb.MessageIdData) { acker.ack(ids) }) tracker.add(&messageID{ledgerID: 1}) tracker.add(&messageID{ledgerID: 2}) @@ -126,7 +133,8 @@ func TestCachedTracker(t *testing.T) { func TestTimedTrackerIndividualAck(t *testing.T) { var acker mockAcker // MaxSize: 1000, MaxTime: 100ms - tracker := newAckGroupingTracker(nil, func(id MessageID) { acker.ack(id) }, nil) + tracker := newAckGroupingTracker(nil, nil, + func(id MessageID) { acker.ackCumulative(id) }, func(ids []*pb.MessageIdData) { acker.ack(ids) }) expected := make([]int64, 0) for i := 0; i < 999; i++ { @@ -161,7 +169,7 @@ func TestTimedTrackerIndividualAck(t *testing.T) { func TestTimedTrackerCumulativeAck(t *testing.T) { var acker mockAcker // MaxTime is 100ms - tracker := newAckGroupingTracker(nil, nil, func(id MessageID) { acker.ackCumulative(id) }) + tracker := newAckGroupingTracker(nil, nil, func(id MessageID) { acker.ackCumulative(id) }, nil) // case 1: flush because of the timeout tracker.addCumulative(&messageID{ledgerID: 1}) @@ -182,7 +190,8 @@ func TestTimedTrackerCumulativeAck(t *testing.T) { } func TestTimedTrackerIsDuplicate(t *testing.T) { - tracker := newAckGroupingTracker(nil, func(id MessageID) {}, func(id MessageID) {}) + tracker := newAckGroupingTracker(nil, func(id MessageID) {}, func(id MessageID) {}, + func(id []*pb.MessageIdData) {}) tracker.add(&messageID{batchIdx: 0, batchSize: 3}) tracker.add(&messageID{batchIdx: 2, batchSize: 3}) @@ -198,8 +207,8 @@ func TestTimedTrackerIsDuplicate(t *testing.T) { func TestDuplicateAfterClose(t *testing.T) { var acker mockAcker - tracker := newAckGroupingTracker(&AckGroupingOptions{MaxSize: 3, MaxTime: 0}, - func(id MessageID) { acker.ack(id) }, func(id MessageID) { acker.ackCumulative(id) }) + tracker := newAckGroupingTracker(&AckGroupingOptions{MaxSize: 3, MaxTime: 0}, nil, + func(id MessageID) { acker.ackCumulative(id) }, func(ids []*pb.MessageIdData) { acker.ack(ids) }) tracker.add(&messageID{ledgerID: 1}) assert.True(t, tracker.isDuplicate(&messageID{ledgerID: 1})) @@ -207,3 +216,28 @@ func TestDuplicateAfterClose(t *testing.T) { tracker.close() assert.False(t, tracker.isDuplicate(&messageID{ledgerID: 1})) } + +func TestTrackerPendingAcks(t *testing.T) { + m := make(map[uint64][]int64) + tracker := newAckGroupingTracker(&AckGroupingOptions{MaxSize: 3, MaxTime: 0}, nil, nil, + func(ids []*pb.MessageIdData) { + for _, id := range ids { + m[*id.LedgerId] = id.AckSet + } + }) + tracker.add(&messageID{ledgerID: 0, batchIdx: 0, batchSize: 30}) + for i := 0; i < 10; i++ { + tracker.add(&messageID{ledgerID: 1, batchIdx: int32(i), batchSize: 10}) + } + assert.Equal(t, 0, len(m)) // the number of entries is 2, so it's not flushed + tracker.flush() + assert.Equal(t, 2, len(m)) + + ackSet, found := m[0] + assert.True(t, found) + assert.Greater(t, len(ackSet), 0) + + ackSet, found = m[1] + assert.True(t, found) + assert.Equal(t, 0, len(ackSet)) // all messages in the batch are acknowledged +} diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index a3dac19c40..18100a9aa9 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -310,7 +310,8 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon pc.unAckChunksTracker = newUnAckChunksTracker(pc) pc.ackGroupingTracker = newAckGroupingTracker(options.ackGroupingOptions, func(id MessageID) { pc.sendIndividualAck(id) }, - func(id MessageID) { pc.sendCumulativeAck(id) }) + func(id MessageID) { pc.sendCumulativeAck(id) }, + func(ids []*pb.MessageIdData) { pc.eventsCh <- ids }) pc.setConsumerState(consumerInit) pc.log = client.log.SubLogger(log.Fields{ "name": pc.name, @@ -837,6 +838,14 @@ func (pc *partitionConsumer) internalAck(req *ackRequest) { } } +func (pc *partitionConsumer) internalAckList(msgIDs []*pb.MessageIdData) { + pc.client.rpcClient.RequestOnCnxNoWait(pc._getConn(), pb.BaseCommand_ACK, &pb.CommandAck{ + AckType: pb.CommandAck_Individual.Enum(), + ConsumerId: proto.Uint64(pc.consumerID), + MessageId: msgIDs, + }) +} + func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, headersAndPayload internal.Buffer) error { pbMsgID := response.GetMessageId() @@ -1364,6 +1373,8 @@ func (pc *partitionConsumer) runEventsLoop() { switch v := i.(type) { case *ackRequest: pc.internalAck(v) + case []*pb.MessageIdData: + pc.internalAckList(v) case *redeliveryRequest: pc.internalRedeliver(v) case *unsubscribeRequest: diff --git a/pulsar/consumer_partition_test.go b/pulsar/consumer_partition_test.go index 16c439978b..21bc0e3a53 100644 --- a/pulsar/consumer_partition_test.go +++ b/pulsar/consumer_partition_test.go @@ -38,7 +38,7 @@ func TestSingleMessageIDNoAckTracker(t *testing.T) { decryptor: crypto.NewNoopDecryptor(), } pc.ackGroupingTracker = newAckGroupingTracker(&AckGroupingOptions{MaxSize: 0}, - func(id MessageID) { pc.sendIndividualAck(id) }, nil) + func(id MessageID) { pc.sendIndividualAck(id) }, nil, nil) headersAndPayload := internal.NewBufferWrapper(rawCompatSingleMessage) if err := pc.MessageReceived(nil, headersAndPayload); err != nil { @@ -76,7 +76,7 @@ func TestBatchMessageIDNoAckTracker(t *testing.T) { decryptor: crypto.NewNoopDecryptor(), } pc.ackGroupingTracker = newAckGroupingTracker(&AckGroupingOptions{MaxSize: 0}, - func(id MessageID) { pc.sendIndividualAck(id) }, nil) + func(id MessageID) { pc.sendIndividualAck(id) }, nil, nil) headersAndPayload := internal.NewBufferWrapper(rawBatchMessage1) if err := pc.MessageReceived(nil, headersAndPayload); err != nil { @@ -111,7 +111,7 @@ func TestBatchMessageIDWithAckTracker(t *testing.T) { decryptor: crypto.NewNoopDecryptor(), } pc.ackGroupingTracker = newAckGroupingTracker(&AckGroupingOptions{MaxSize: 0}, - func(id MessageID) { pc.sendIndividualAck(id) }, nil) + func(id MessageID) { pc.sendIndividualAck(id) }, nil, nil) headersAndPayload := internal.NewBufferWrapper(rawBatchMessage10) if err := pc.MessageReceived(nil, headersAndPayload); err != nil { diff --git a/pulsar/message.go b/pulsar/message.go index c44957d18d..98190e918b 100644 --- a/pulsar/message.go +++ b/pulsar/message.go @@ -212,11 +212,3 @@ func messageIDCompare(lhs MessageID, rhs MessageID) int { } return 0 } - -func messageIDHash(id MessageID) int64 { - return id.LedgerID() + 31*id.EntryID() -} - -func messageIDIsBatch(id MessageID) bool { - return id.BatchIdx() >= 0 && id.BatchSize() > 0 -} From bcbac9f2ae5c82c6b995cbfb8092f2a96c3aeb14 Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Mon, 13 Mar 2023 11:01:01 +0800 Subject: [PATCH 034/348] [perf]: More precise producer rate limiter (#989) * perf: more accurate producer rate limiter * perf: more accurate producer rate limiter --- go.mod | 1 + go.sum | 1 + perf/perf-producer.go | 17 ++++++----------- 3 files changed, 8 insertions(+), 11 deletions(-) diff --git a/go.mod b/go.mod index fe6f41556b..3e7299b07b 100644 --- a/go.mod +++ b/go.mod @@ -26,6 +26,7 @@ require ( go.uber.org/atomic v1.7.0 golang.org/x/mod v0.5.1 golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 + golang.org/x/time v0.0.0-20191024005414-555d28b269f0 google.golang.org/protobuf v1.26.0 ) diff --git a/go.sum b/go.sum index 4e96e3988e..528c45882f 100644 --- a/go.sum +++ b/go.sum @@ -427,6 +427,7 @@ golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0 h1:/5xXl8Y5W96D+TtHSlonuFqGHIWVuyCkGJLwGh9JJFs= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/perf/perf-producer.go b/perf/perf-producer.go index 9d062ae105..f8e14d22dc 100644 --- a/perf/perf-producer.go +++ b/perf/perf-producer.go @@ -22,6 +22,8 @@ import ( "encoding/json" "time" + "golang.org/x/time/rate" + "github.com/bmizerany/perks/quantile" "github.com/spf13/cobra" @@ -104,16 +106,9 @@ func produce(produceArgs *ProduceArgs, stop <-chan struct{}) { payload := make([]byte, produceArgs.MessageSize) ch := make(chan float64) - rateLimitCh := make(chan time.Time, produceArgs.Rate) - go func(rateLimit int, interval time.Duration) { - if rateLimit <= 0 { // 0 as no limit enforced - return - } - for { - oldest := <-rateLimitCh - time.Sleep(interval - time.Since(oldest)) - } - }(produceArgs.Rate, time.Second) + + limit := rate.Every(time.Duration(float64(time.Second) / float64(produceArgs.Rate))) + rateLimiter := rate.NewLimiter(limit, produceArgs.Rate) go func(stopCh <-chan struct{}) { for { @@ -125,7 +120,7 @@ func produce(produceArgs *ProduceArgs, stop <-chan struct{}) { start := time.Now() if produceArgs.Rate > 0 { - rateLimitCh <- start + _ = rateLimiter.Wait(context.TODO()) } producer.SendAsync(ctx, &pulsar.ProducerMessage{ From 257a9c84cc4c255e4c8ceb5a77f6abcbd927ff09 Mon Sep 17 00:00:00 2001 From: Lei Zhiyuan Date: Wed, 15 Mar 2023 15:26:56 +0800 Subject: [PATCH 035/348] fix:when send error, nack the message (#592) --- pulsar/dlq_router.go | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/pulsar/dlq_router.go b/pulsar/dlq_router.go index 5ecd8f8632..6d30003640 100644 --- a/pulsar/dlq_router.go +++ b/pulsar/dlq_router.go @@ -109,13 +109,17 @@ func (r *dlqRouter) run() { Properties: properties, EventTime: msg.EventTime(), ReplicationClusters: msg.replicationClusters, - }, func(MessageID, *ProducerMessage, error) { - r.log.WithField("msgID", msgID).Debug("Sent message to DLQ") - - // The Producer ack might be coming from the connection go-routine that - // is also used by the consumer. In that case we would get a dead-lock - // if we'd try to ack. - go cm.Consumer.AckID(msgID) + }, func(messageID MessageID, producerMessage *ProducerMessage, err error) { + if err == nil { + r.log.WithField("msgID", msgID).Debug("Succeed to send message to DLQ") + // The Producer ack might be coming from the connection go-routine that + // is also used by the consumer. In that case we would get a dead-lock + // if we'd try to ack. + go cm.Consumer.AckID(msgID) + } else { + r.log.WithError(err).WithField("msgID", msgID).Debug("Failed to send message to DLQ") + go cm.Consumer.Nack(cm) + } }) case <-r.closeCh: From b8563cd809027ba85ce66c8c75621243e969864e Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Wed, 15 Mar 2023 15:28:48 +0800 Subject: [PATCH 036/348] [feat]: Support auto scaled consumer receiver queue (#976) * feat: support auto scaled receiver queue --- perf/perf-consumer.go | 18 +++-- pulsar/consumer.go | 6 ++ pulsar/consumer_impl.go | 1 + pulsar/consumer_partition.go | 113 +++++++++++++++++++++++++----- pulsar/consumer_partition_test.go | 3 + pulsar/consumer_test.go | 85 ++++++++++++++++++++++ 6 files changed, 203 insertions(+), 23 deletions(-) diff --git a/perf/perf-consumer.go b/perf/perf-consumer.go index 6b6e411dbc..825de62ff0 100644 --- a/perf/perf-consumer.go +++ b/perf/perf-consumer.go @@ -31,10 +31,11 @@ import ( // ConsumeArgs define the parameters required by consume type ConsumeArgs struct { - Topic string - SubscriptionName string - ReceiverQueueSize int - EnableBatchIndexAck bool + Topic string + SubscriptionName string + ReceiverQueueSize int + EnableBatchIndexAck bool + EnableAutoScaledReceiverQueueSize bool } func newConsumerCommand() *cobra.Command { @@ -57,6 +58,8 @@ func newConsumerCommand() *cobra.Command { flags.StringVarP(&consumeArgs.SubscriptionName, "subscription", "s", "sub", "Subscription name") flags.IntVarP(&consumeArgs.ReceiverQueueSize, "receiver-queue-size", "r", 1000, "Receiver queue size") flags.BoolVar(&consumeArgs.EnableBatchIndexAck, "enable-batch-index-ack", false, "Whether to enable batch index ACK") + flags.BoolVar(&consumeArgs.EnableAutoScaledReceiverQueueSize, "enable-auto-scaled-queue-size", false, + "Whether to enable auto scaled receiver queue size") return cmd } @@ -76,9 +79,10 @@ func consume(consumeArgs *ConsumeArgs, stop <-chan struct{}) { defer client.Close() consumer, err := client.Subscribe(pulsar.ConsumerOptions{ - Topic: consumeArgs.Topic, - SubscriptionName: consumeArgs.SubscriptionName, - EnableBatchIndexAcknowledgment: consumeArgs.EnableBatchIndexAck, + Topic: consumeArgs.Topic, + SubscriptionName: consumeArgs.SubscriptionName, + EnableBatchIndexAcknowledgment: consumeArgs.EnableBatchIndexAck, + EnableAutoScaledReceiverQueueSize: consumeArgs.EnableAutoScaledReceiverQueueSize, }) if err != nil { diff --git a/pulsar/consumer.go b/pulsar/consumer.go index 9576d7aabc..64a096d50c 100644 --- a/pulsar/consumer.go +++ b/pulsar/consumer.go @@ -160,6 +160,12 @@ type ConsumerOptions struct { // Default value is `1000` messages and should be good for most use cases. ReceiverQueueSize int + // EnableAutoScaledReceiverQueueSize, if enabled, the consumer receive queue will be auto-scaled + // by the consumer actual throughput. The ReceiverQueueSize will be the maximum size which consumer + // receive queue can be scaled. + // Default is false. + EnableAutoScaledReceiverQueueSize bool + // NackRedeliveryDelay specifies the delay after which to redeliver the messages that failed to be // processed. Default is 1 min. (See `Consumer.Nack()`) NackRedeliveryDelay time.Duration diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index d19e52237f..fd7fa5781d 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -399,6 +399,7 @@ func (c *consumer) internalTopicSubscribeToPartitions() error { consumerEventListener: c.options.EventListener, enableBatchIndexAck: c.options.EnableBatchIndexAcknowledgment, ackGroupingOptions: c.options.AckGroupingOptions, + autoReceiverQueueSize: c.options.EnableAutoScaledReceiverQueueSize, } cons, err := newPartitionConsumer(c, c.client, opts, c.messageCh, c.dlq, c.metrics) ch <- ConsumerError{ diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 18100a9aa9..a08405a838 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -25,7 +25,6 @@ import ( "math" "strings" "sync" - "sync/atomic" "time" "google.golang.org/protobuf/proto" @@ -77,6 +76,10 @@ const ( nonDurable ) +const ( + initialReceiverQueueSize = 1 +) + const ( noMessageEntry = -1 ) @@ -89,6 +92,7 @@ type partitionConsumerOpts struct { subscriptionInitPos SubscriptionInitialPosition partitionIdx int receiverQueueSize int + autoReceiverQueueSize bool nackRedeliveryDelay time.Duration nackBackoffPolicy NackBackoffPolicy metadata map[string]string @@ -142,11 +146,15 @@ type partitionConsumer struct { availablePermits *availablePermits // the size of the queue channel for buffering messages - queueSize int32 + maxQueueSize int32 queueCh chan []*message startMessageID atomicMessageID lastDequeuedMsg *trackingMessageID + currentQueueSize uAtomic.Int32 + scaleReceiverQueueHint uAtomic.Bool + incomingMessages uAtomic.Int32 + eventsCh chan interface{} connectedCh chan struct{} connectClosedCh chan connectionClosed @@ -181,22 +189,44 @@ func (pc *partitionConsumer) ActiveConsumerChanged(isActive bool) { } type availablePermits struct { - permits int32 + permits uAtomic.Int32 pc *partitionConsumer } func (p *availablePermits) inc() { // atomic add availablePermits - ap := atomic.AddInt32(&p.permits, 1) + p.add(1) +} + +func (p *availablePermits) add(delta int32) { + p.permits.Add(delta) + p.flowIfNeed() +} + +func (p *availablePermits) reset() { + p.permits.Store(0) +} +func (p *availablePermits) get() int32 { + return p.permits.Load() +} + +func (p *availablePermits) flowIfNeed() { // TODO implement a better flow controller // send more permits if needed - flowThreshold := int32(math.Max(float64(p.pc.queueSize/2), 1)) - if ap >= flowThreshold { - availablePermits := ap - requestedPermits := ap + var flowThreshold int32 + if p.pc.options.autoReceiverQueueSize { + flowThreshold = int32(math.Max(float64(p.pc.currentQueueSize.Load()/2), 1)) + } else { + flowThreshold = int32(math.Max(float64(p.pc.maxQueueSize/2), 1)) + } + + current := p.get() + if current >= flowThreshold { + availablePermits := current + requestedPermits := current // check if permits changed - if !atomic.CompareAndSwapInt32(&p.permits, ap, 0) { + if !p.permits.CAS(current, 0) { return } @@ -207,10 +237,6 @@ func (p *availablePermits) inc() { } } -func (p *availablePermits) reset() { - atomic.StoreInt32(&p.permits, 0) -} - // atomicMessageID is a wrapper for trackingMessageID to make get and set atomic type atomicMessageID struct { msgID *trackingMessageID @@ -292,7 +318,7 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon consumerID: client.rpcClient.NewConsumerID(), partitionIdx: int32(options.partitionIdx), eventsCh: make(chan interface{}, 10), - queueSize: int32(options.receiverQueueSize), + maxQueueSize: int32(options.receiverQueueSize), queueCh: make(chan []*message, options.receiverQueueSize), startMessageID: atomicMessageID{msgID: options.startMessageID}, connectedCh: make(chan struct{}), @@ -305,6 +331,11 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon metrics: metrics, schemaInfoCache: newSchemaInfoCache(client, options.topic), } + if pc.options.autoReceiverQueueSize { + pc.currentQueueSize.Store(initialReceiverQueueSize) + } else { + pc.currentQueueSize.Store(int32(pc.options.receiverQueueSize)) + } pc.availablePermits = &availablePermits{pc: pc} pc.chunkedMsgCtxMap = newChunkedMsgCtxMap(options.maxPendingChunkedMessage, pc) pc.unAckChunksTracker = newUnAckChunksTracker(pc) @@ -904,6 +935,12 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header orderingKey: string(msgMeta.OrderingKey), }, } + + if pc.options.autoReceiverQueueSize { + pc.incomingMessages.Inc() + pc.markScaleIfNeed() + } + pc.queueCh <- messages return nil } @@ -1073,6 +1110,11 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header messages = append(messages, msg) } + if pc.options.autoReceiverQueueSize { + pc.incomingMessages.Add(int32(len(messages))) + pc.markScaleIfNeed() + } + // send messages to the dispatcher pc.queueCh <- messages return nil @@ -1240,7 +1282,6 @@ func (pc *partitionConsumer) dispatcher() { pc.metrics.PrefetchedMessages.Dec() pc.metrics.PrefetchedBytes.Sub(float64(len(messages[0].payLoad))) } else { - // we are ready for more messages queueCh = pc.queueCh } @@ -1258,7 +1299,13 @@ func (pc *partitionConsumer) dispatcher() { // reset available permits pc.availablePermits.reset() - initialPermits := uint32(pc.queueSize) + + var initialPermits uint32 + if pc.options.autoReceiverQueueSize { + initialPermits = uint32(pc.currentQueueSize.Load()) + } else { + initialPermits = uint32(pc.maxQueueSize) + } pc.log.Debugf("dispatcher requesting initial permits=%d", initialPermits) // send initial permits @@ -1282,6 +1329,11 @@ func (pc *partitionConsumer) dispatcher() { pc.availablePermits.inc() + if pc.options.autoReceiverQueueSize { + pc.incomingMessages.Dec() + pc.expectMoreIncomingMessages() + } + case clearQueueCb := <-pc.clearQueueCh: // drain the message queue on any new connection by sending a // special nil message to the channel so we know when to stop dropping messages @@ -1297,6 +1349,9 @@ func (pc *partitionConsumer) dispatcher() { } else if nextMessageInQueue == nil { nextMessageInQueue = toTrackingMessageID(m[0].msgID) } + if pc.options.autoReceiverQueueSize { + pc.incomingMessages.Sub(int32(len(m))) + } } messages = nil @@ -1671,6 +1726,32 @@ func getPreviousMessage(mid *trackingMessageID) *trackingMessageID { } } +func (pc *partitionConsumer) expectMoreIncomingMessages() { + if !pc.options.autoReceiverQueueSize { + return + } + if pc.scaleReceiverQueueHint.CAS(true, false) { + oldSize := pc.currentQueueSize.Load() + maxSize := int32(pc.options.receiverQueueSize) + newSize := int32(math.Min(float64(maxSize), float64(oldSize*2))) + if newSize > oldSize { + pc.currentQueueSize.CAS(oldSize, newSize) + pc.availablePermits.add(newSize - oldSize) + pc.log.Debugf("update currentQueueSize from %d -> %d", oldSize, newSize) + } + } +} + +func (pc *partitionConsumer) markScaleIfNeed() { + // availablePermits + incomingMessages (messages in queueCh) is the number of prefetched messages + // The result of auto-scale we expected is currentQueueSize is slightly bigger than prefetched messages + prev := pc.scaleReceiverQueueHint.Swap(pc.availablePermits.get()+pc.incomingMessages.Load() >= + pc.currentQueueSize.Load()) + if prev != pc.scaleReceiverQueueHint.Load() { + pc.log.Debugf("update scaleReceiverQueueHint from %t -> %t", prev, pc.scaleReceiverQueueHint.Load()) + } +} + func (pc *partitionConsumer) Decompress(msgMeta *pb.MessageMetadata, payload internal.Buffer) (internal.Buffer, error) { providerEntry, ok := pc.compressionProviders.Load(msgMeta.GetCompression()) if !ok { diff --git a/pulsar/consumer_partition_test.go b/pulsar/consumer_partition_test.go index 21bc0e3a53..21280fb781 100644 --- a/pulsar/consumer_partition_test.go +++ b/pulsar/consumer_partition_test.go @@ -37,6 +37,7 @@ func TestSingleMessageIDNoAckTracker(t *testing.T) { metrics: newTestMetrics(), decryptor: crypto.NewNoopDecryptor(), } + pc.availablePermits = &availablePermits{pc: &pc} pc.ackGroupingTracker = newAckGroupingTracker(&AckGroupingOptions{MaxSize: 0}, func(id MessageID) { pc.sendIndividualAck(id) }, nil, nil) @@ -75,6 +76,7 @@ func TestBatchMessageIDNoAckTracker(t *testing.T) { metrics: newTestMetrics(), decryptor: crypto.NewNoopDecryptor(), } + pc.availablePermits = &availablePermits{pc: &pc} pc.ackGroupingTracker = newAckGroupingTracker(&AckGroupingOptions{MaxSize: 0}, func(id MessageID) { pc.sendIndividualAck(id) }, nil, nil) @@ -110,6 +112,7 @@ func TestBatchMessageIDWithAckTracker(t *testing.T) { metrics: newTestMetrics(), decryptor: crypto.NewNoopDecryptor(), } + pc.availablePermits = &availablePermits{pc: &pc} pc.ackGroupingTracker = newAckGroupingTracker(&AckGroupingOptions{MaxSize: 0}, func(id MessageID) { pc.sendIndividualAck(id) }, nil, nil) diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 21fa7d04d4..a93dea9f35 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -3988,3 +3988,88 @@ func runBatchIndexAckTest(t *testing.T, ackWithResponse bool, cumulative bool, o client.Close() } + +func TestConsumerWithAutoScaledQueueReceive(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + + topic := newTopicName() + + // create consumer + c, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + Type: Exclusive, + ReceiverQueueSize: 3, + EnableAutoScaledReceiverQueueSize: true, + }) + assert.Nil(t, err) + pc := c.(*consumer).consumers[0] + assert.Equal(t, int32(1), pc.currentQueueSize.Load()) + defer c.Close() + + // create p + p, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + DisableBatching: false, + }) + assert.Nil(t, err) + defer p.Close() + + // send message, it will update scaleReceiverQueueHint from false to true + _, err = p.Send(context.Background(), &ProducerMessage{ + Payload: []byte("hello"), + }) + assert.NoError(t, err) + + // this will trigger receiver queue size expanding to 2 because we have prefetched 1 message >= currentSize 1. + _, err = c.Receive(context.Background()) + assert.Nil(t, err) + + // currentQueueSize should be doubled in size + retryAssert(t, 5, 200, func() {}, func(t assert.TestingT) bool { + return assert.Equal(t, 2, int(pc.currentQueueSize.Load())) + }) + + for i := 0; i < 5; i++ { + _, err = p.Send(context.Background(), &ProducerMessage{ + Payload: []byte("hello"), + }) + assert.NoError(t, err) + + // waiting for prefetched message passing from queueCh to messageCh + retryAssert(t, 5, 200, func() {}, func(t assert.TestingT) bool { + return assert.Equal(t, 1, len(pc.messageCh)) + }) + + _, err = p.Send(context.Background(), &ProducerMessage{ + Payload: []byte("hello"), + }) + assert.NoError(t, err) + + // wait all the messages has been prefetched + _, err = c.Receive(context.Background()) + assert.Nil(t, err) + _, err = c.Receive(context.Background()) + assert.Nil(t, err) + // this will not trigger receiver queue size expanding because we have prefetched 2 message < currentSize 4. + assert.Equal(t, int32(2), pc.currentQueueSize.Load()) + } + + for i := 0; i < 5; i++ { + p.SendAsync( + context.Background(), + &ProducerMessage{Payload: []byte("hello")}, + func(id MessageID, producerMessage *ProducerMessage, err error) { + }, + ) + } + + retryAssert(t, 3, 300, func() {}, func(t assert.TestingT) bool { + return assert.Equal(t, 3, int(pc.currentQueueSize.Load())) + }) +} From 04ad521e6d061768ff02ac9bf024932d45b2bd94 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 20 Mar 2023 22:18:16 +0800 Subject: [PATCH 037/348] Fix batched messages not ACKed correctly when batch index ACK is disabled (#994) * Fix batched messages not ACKed correctly when batch index ACK is disabled Fixes /~https://github.com/apache/pulsar-client-go/issues/993 ### Motivation When batch index ACK is disabled, if N messages in a batch are acknowledged, currently only the batched message ID of the last message will be acknowledged. This behavior is wrong because we need to acknowledge the whole batch. ### Modifications - Create a `messageID` instance to ACK for this case - Add `TestConsumerBatchIndexAckDisabled` to cover this case * Fix ackRequest error * Fix wrong received msg id * Add comments --- pulsar/consumer_partition.go | 8 ++++++++ pulsar/consumer_test.go | 40 ++++++++++++++++++++++++++++++++++++ 2 files changed, 48 insertions(+) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index a08405a838..087cec550e 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -499,6 +499,14 @@ func (pc *partitionConsumer) ackID(msgID MessageID, withResponse bool) error { trackingID := toTrackingMessageID(msgID) if trackingID != nil && trackingID.ack() { + // All messages in the same batch have been acknowledged, we only need to acknowledge the + // MessageID that represents the entry that stores the whole batch + trackingID = &trackingMessageID{ + messageID: &messageID{ + ledgerID: trackingID.ledgerID, + entryID: trackingID.entryID, + }, + } pc.metrics.AcksCounter.Inc() pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9) } else if !pc.options.enableBatchIndexAck { diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index a93dea9f35..1477ce6c20 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -4073,3 +4073,43 @@ func TestConsumerWithAutoScaledQueueReceive(t *testing.T) { return assert.Equal(t, 3, int(pc.currentQueueSize.Load())) }) } + +func TestConsumerBatchIndexAckDisabled(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + assert.Nil(t, err) + defer client.Close() + + topic := newTopicName() + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + }) + assert.Nil(t, err) + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + }) + assert.Nil(t, err) + + for i := 0; i < 5; i++ { + producer.SendAsync(context.Background(), &ProducerMessage{ + Payload: []byte(fmt.Sprintf("msg-%d", i)), + }, nil) + } + for i := 0; i < 5; i++ { + message, err := consumer.Receive(context.Background()) + assert.Nil(t, err) + consumer.Ack(message) + } + consumer.Close() + consumer, err = client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + }) + assert.Nil(t, err) + producer.Send(context.Background(), &ProducerMessage{Payload: []byte("done")}) + message, err := consumer.Receive(context.Background()) + assert.Nil(t, err) + assert.Equal(t, []byte("done"), message.Payload()) +} From 6c3ee77e453a028f4d2dadbb41c357d90aa8d813 Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Wed, 22 Mar 2023 10:35:04 +0800 Subject: [PATCH 038/348] fix: fix retryAssert (#996) --- pulsar/helper_for_test.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/pulsar/helper_for_test.go b/pulsar/helper_for_test.go index eda2d72b4e..7bbf66e9fd 100644 --- a/pulsar/helper_for_test.go +++ b/pulsar/helper_for_test.go @@ -177,9 +177,14 @@ func retryAssert(t assert.TestingT, times int, milliseconds int, update func(), for i := 0; i < times; i++ { time.Sleep(time.Duration(milliseconds) * time.Millisecond) update() - if assert(nil) { + if assert(fakeAssertT{}) { break } } assert(t) } + +type fakeAssertT struct{} + +func (fa fakeAssertT) Errorf(format string, args ...interface{}) { +} From 20291f5cf5d5ea706e6a913c5d9eb303fedaef0d Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Thu, 23 Mar 2023 16:54:58 +0800 Subject: [PATCH 039/348] [feat] Support consumer client memory limit (#991) * feat: support consumer memory limit * fix: avoid data race * fix: shrinking triggered by memlimit * fix: fix flaky test * fix: fix flaky test * fix: modify trigger threshold * fix: fix memory limit controller unit test * fix: fix setRunning * fix: fix TestRegisterTrigger and TestMultiConsumerMemoryLimit --- pulsar/client_impl.go | 15 +- pulsar/consumer_partition.go | 28 ++- pulsar/consumer_test.go | 201 ++++++++++++++++++ pulsar/impl_message.go | 4 + pulsar/internal/memory_limit_controller.go | 58 ++++- .../internal/memory_limit_controller_test.go | 63 +++++- 6 files changed, 350 insertions(+), 19 deletions(-) diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index 7c8fcc9cf4..5322597569 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -30,12 +30,13 @@ import ( ) const ( - defaultConnectionTimeout = 10 * time.Second - defaultOperationTimeout = 30 * time.Second - defaultKeepAliveInterval = 30 * time.Second - defaultMemoryLimitBytes = 64 * 1024 * 1024 - defaultConnMaxIdleTime = 180 * time.Second - minConnMaxIdleTime = 60 * time.Second + defaultConnectionTimeout = 10 * time.Second + defaultOperationTimeout = 30 * time.Second + defaultKeepAliveInterval = 30 * time.Second + defaultMemoryLimitBytes = 64 * 1024 * 1024 + defaultMemoryLimitTriggerThreshold = 0.95 + defaultConnMaxIdleTime = 180 * time.Second + minConnMaxIdleTime = 60 * time.Second ) type client struct { @@ -158,7 +159,7 @@ func newClient(options ClientOptions) (Client, error) { maxConnectionsPerHost, logger, metrics, connectionMaxIdleTime), log: logger, metrics: metrics, - memLimit: internal.NewMemoryLimitController(memLimitBytes), + memLimit: internal.NewMemoryLimitController(memLimitBytes, defaultMemoryLimitTriggerThreshold), } serviceNameResolver := internal.NewPulsarServiceNameResolver(url) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 087cec550e..fb77d0dc6a 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -77,7 +77,8 @@ const ( ) const ( - initialReceiverQueueSize = 1 + initialReceiverQueueSize = 1 + receiverQueueExpansionMemThreshold = 0.75 ) const ( @@ -333,6 +334,7 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon } if pc.options.autoReceiverQueueSize { pc.currentQueueSize.Store(initialReceiverQueueSize) + pc.client.memLimit.RegisterTrigger(pc.shrinkReceiverQueueSize) } else { pc.currentQueueSize.Store(int32(pc.options.receiverQueueSize)) } @@ -1002,6 +1004,7 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header pc.metrics.MessagesReceived.Add(float64(numMsgs)) pc.metrics.PrefetchedMessages.Add(float64(numMsgs)) + var bytesReceived int for i := 0; i < numMsgs; i++ { smm, payload, err := reader.ReadMessage() if err != nil || payload == nil { @@ -1116,9 +1119,11 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header }) messages = append(messages, msg) + bytesReceived += msg.size() } if pc.options.autoReceiverQueueSize { + pc.client.memLimit.ForceReserveMemory(int64(bytesReceived)) pc.incomingMessages.Add(int32(len(messages))) pc.markScaleIfNeed() } @@ -1270,6 +1275,7 @@ func (pc *partitionConsumer) dispatcher() { var queueCh chan []*message var messageCh chan ConsumerMessage var nextMessage ConsumerMessage + var nextMessageSize int // are there more messages to send? if len(messages) > 0 { @@ -1277,6 +1283,7 @@ func (pc *partitionConsumer) dispatcher() { Consumer: pc.parentConsumer, Message: messages[0], } + nextMessageSize = messages[0].size() if pc.dlq.shouldSendToDlq(&nextMessage) { // pass the message to the DLQ router @@ -1339,6 +1346,7 @@ func (pc *partitionConsumer) dispatcher() { if pc.options.autoReceiverQueueSize { pc.incomingMessages.Dec() + pc.client.memLimit.ReleaseMemory(int64(nextMessageSize)) pc.expectMoreIncomingMessages() } @@ -1742,7 +1750,8 @@ func (pc *partitionConsumer) expectMoreIncomingMessages() { oldSize := pc.currentQueueSize.Load() maxSize := int32(pc.options.receiverQueueSize) newSize := int32(math.Min(float64(maxSize), float64(oldSize*2))) - if newSize > oldSize { + usagePercent := pc.client.memLimit.CurrentUsagePercent() + if usagePercent < receiverQueueExpansionMemThreshold && newSize > oldSize { pc.currentQueueSize.CAS(oldSize, newSize) pc.availablePermits.add(newSize - oldSize) pc.log.Debugf("update currentQueueSize from %d -> %d", oldSize, newSize) @@ -1760,6 +1769,21 @@ func (pc *partitionConsumer) markScaleIfNeed() { } } +func (pc *partitionConsumer) shrinkReceiverQueueSize() { + if !pc.options.autoReceiverQueueSize { + return + } + + oldSize := pc.currentQueueSize.Load() + minSize := int32(math.Min(float64(initialReceiverQueueSize), float64(pc.options.receiverQueueSize))) + newSize := int32(math.Max(float64(minSize), float64(oldSize/2))) + if newSize < oldSize { + pc.currentQueueSize.CAS(oldSize, newSize) + pc.availablePermits.add(newSize - oldSize) + pc.log.Debugf("update currentQueueSize from %d -> %d", oldSize, newSize) + } +} + func (pc *partitionConsumer) Decompress(msgMeta *pb.MessageMetadata, payload internal.Buffer) (internal.Buffer, error) { providerEntry, ok := pc.compressionProviders.Load(msgMeta.GetCompression()) if !ok { diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 1477ce6c20..521e576767 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -4113,3 +4113,204 @@ func TestConsumerBatchIndexAckDisabled(t *testing.T) { assert.Nil(t, err) assert.Equal(t, []byte("done"), message.Payload()) } + +func TestConsumerMemoryLimit(t *testing.T) { + // Create client 1 without memory limit + cli1, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer cli1.Close() + + // Create client 1 with memory limit + cli2, err := NewClient(ClientOptions{ + URL: lookupURL, + MemoryLimitBytes: 10 * 1024, + }) + + assert.Nil(t, err) + defer cli2.Close() + + topic := newTopicName() + + // Use client 1 to create producer p1 + p1, err := cli1.CreateProducer(ProducerOptions{ + Topic: topic, + DisableBatching: false, + }) + assert.Nil(t, err) + defer p1.Close() + + // Use mem-limited client 2 to create consumer c1 + c1, err := cli2.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub-1", + Type: Exclusive, + EnableAutoScaledReceiverQueueSize: true, + }) + assert.Nil(t, err) + defer c1.Close() + pc1 := c1.(*consumer).consumers[0] + + // Fill up the messageCh of c1 + for i := 0; i < 10; i++ { + p1.SendAsync( + context.Background(), + &ProducerMessage{Payload: createTestMessagePayload(1)}, + func(id MessageID, producerMessage *ProducerMessage, err error) { + }, + ) + } + + retryAssert(t, 5, 200, func() {}, func(t assert.TestingT) bool { + return assert.Equal(t, 10, len(pc1.messageCh)) + }) + + // Get current receiver queue size of c1 + prevQueueSize := pc1.currentQueueSize.Load() + + // Make the client 1 exceed the memory limit + _, err = p1.Send(context.Background(), &ProducerMessage{ + Payload: createTestMessagePayload(10*1024 + 1), + }) + assert.NoError(t, err) + + // c1 should shrink it's receiver queue size + retryAssert(t, 5, 200, func() {}, func(t assert.TestingT) bool { + return assert.Equal(t, prevQueueSize/2, pc1.currentQueueSize.Load()) + }) + + // Use mem-limited client 2 to create consumer c2 + c2, err := cli2.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub-2", + Type: Exclusive, + SubscriptionInitialPosition: SubscriptionPositionEarliest, + EnableAutoScaledReceiverQueueSize: true, + }) + assert.Nil(t, err) + defer c2.Close() + pc2 := c2.(*consumer).consumers[0] + + // Try to induce c2 receiver queue size expansion + for i := 0; i < 10; i++ { + p1.SendAsync( + context.Background(), + &ProducerMessage{Payload: createTestMessagePayload(1)}, + func(id MessageID, producerMessage *ProducerMessage, err error) { + }, + ) + } + + retryAssert(t, 5, 200, func() {}, func(t assert.TestingT) bool { + return assert.Equal(t, 10, len(pc1.messageCh)) + }) + + // c2 receiver queue size should not expansion because client 1 has exceeded the memory limit + assert.Equal(t, 1, int(pc2.currentQueueSize.Load())) + + // Use mem-limited client 2 to create producer p2 + p2, err := cli2.CreateProducer(ProducerOptions{ + Topic: topic, + DisableBatching: false, + DisableBlockIfQueueFull: true, + }) + assert.Nil(t, err) + defer p2.Close() + + _, err = p2.Send(context.Background(), &ProducerMessage{ + Payload: createTestMessagePayload(1), + }) + // Producer can't send message + assert.Equal(t, true, errors.Is(err, errMemoryBufferIsFull)) +} + +func TestMultiConsumerMemoryLimit(t *testing.T) { + // Create client 1 without memory limit + cli1, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer cli1.Close() + + // Create client 1 with memory limit + cli2, err := NewClient(ClientOptions{ + URL: lookupURL, + MemoryLimitBytes: 10 * 1024, + }) + + assert.Nil(t, err) + defer cli2.Close() + + topic := newTopicName() + + // Use client 1 to create producer p1 + p1, err := cli1.CreateProducer(ProducerOptions{ + Topic: topic, + DisableBatching: false, + }) + assert.Nil(t, err) + defer p1.Close() + + // Use mem-limited client 2 to create consumer c1 + c1, err := cli2.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub-1", + Type: Exclusive, + EnableAutoScaledReceiverQueueSize: true, + }) + assert.Nil(t, err) + defer c1.Close() + pc1 := c1.(*consumer).consumers[0] + + // Use mem-limited client 2 to create consumer c1 + c2, err := cli2.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub-2", + Type: Exclusive, + EnableAutoScaledReceiverQueueSize: true, + }) + assert.Nil(t, err) + defer c2.Close() + pc2 := c2.(*consumer).consumers[0] + + // Fill up the messageCh of c1 nad c2 + for i := 0; i < 10; i++ { + p1.SendAsync( + context.Background(), + &ProducerMessage{Payload: createTestMessagePayload(1)}, + func(id MessageID, producerMessage *ProducerMessage, err error) { + }, + ) + } + + retryAssert(t, 5, 200, func() {}, func(t assert.TestingT) bool { + return assert.Equal(t, 10, len(pc1.messageCh)) + }) + + retryAssert(t, 5, 200, func() {}, func(t assert.TestingT) bool { + return assert.Equal(t, 10, len(pc2.messageCh)) + }) + + // Get current receiver queue size of c1 and c2 + pc1PrevQueueSize := pc1.currentQueueSize.Load() + pc2PrevQueueSize := pc2.currentQueueSize.Load() + + // Make the client 1 exceed the memory limit + _, err = p1.Send(context.Background(), &ProducerMessage{ + Payload: createTestMessagePayload(10*1024 + 1), + }) + assert.NoError(t, err) + + // c1 should shrink it's receiver queue size + retryAssert(t, 5, 200, func() {}, func(t assert.TestingT) bool { + return assert.Equal(t, pc1PrevQueueSize/2, pc1.currentQueueSize.Load()) + }) + + // c2 should shrink it's receiver queue size too + retryAssert(t, 5, 200, func() {}, func(t assert.TestingT) bool { + return assert.Equal(t, pc2PrevQueueSize/2, pc2.currentQueueSize.Load()) + }) +} diff --git a/pulsar/impl_message.go b/pulsar/impl_message.go index 9c56070295..89a709f14a 100644 --- a/pulsar/impl_message.go +++ b/pulsar/impl_message.go @@ -382,6 +382,10 @@ func (msg *message) BrokerPublishTime() *time.Time { return msg.brokerPublishTime } +func (msg *message) size() int { + return len(msg.payLoad) +} + func newAckTracker(size uint) *ackTracker { batchIDs := bitset.New(size) for i := uint(0); i < size; i++ { diff --git a/pulsar/internal/memory_limit_controller.go b/pulsar/internal/memory_limit_controller.go index 5bf8d59a58..5ead9f5b9a 100644 --- a/pulsar/internal/memory_limit_controller.go +++ b/pulsar/internal/memory_limit_controller.go @@ -31,18 +31,42 @@ type MemoryLimitController interface { CurrentUsage() int64 CurrentUsagePercent() float64 IsMemoryLimited() bool + RegisterTrigger(trigger func()) } type memoryLimitController struct { limit int64 chCond *chCond currentUsage int64 + + triggers []*thresholdTrigger + // valid range is (0, 1.0) + triggerThreshold float64 +} + +type thresholdTrigger struct { + triggerFunc func() + triggerRunning int32 +} + +func (t *thresholdTrigger) canTryRunning() bool { + return atomic.CompareAndSwapInt32(&t.triggerRunning, 0, 1) +} + +func (t *thresholdTrigger) setRunning(isRunning bool) { + if isRunning { + atomic.StoreInt32(&t.triggerRunning, 1) + } else { + atomic.StoreInt32(&t.triggerRunning, 0) + } } -func NewMemoryLimitController(limit int64) MemoryLimitController { +// NewMemoryLimitController threshold valid range is (0, 1.0) +func NewMemoryLimitController(limit int64, threshold float64) MemoryLimitController { mlc := &memoryLimitController{ - limit: limit, - chCond: newCond(&sync.Mutex{}), + limit: limit, + chCond: newCond(&sync.Mutex{}), + triggerThreshold: threshold, } return mlc } @@ -72,13 +96,16 @@ func (m *memoryLimitController) TryReserveMemory(size int64) bool { } if atomic.CompareAndSwapInt64(&m.currentUsage, current, newUsage) { + m.checkTrigger(current, newUsage) return true } } } func (m *memoryLimitController) ForceReserveMemory(size int64) { - atomic.AddInt64(&m.currentUsage, size) + nextUsage := atomic.AddInt64(&m.currentUsage, size) + prevUsage := nextUsage - size + m.checkTrigger(prevUsage, nextUsage) } func (m *memoryLimitController) ReleaseMemory(size int64) { @@ -99,3 +126,26 @@ func (m *memoryLimitController) CurrentUsagePercent() float64 { func (m *memoryLimitController) IsMemoryLimited() bool { return m.limit > 0 } + +func (m *memoryLimitController) RegisterTrigger(trigger func()) { + m.chCond.L.Lock() + defer m.chCond.L.Unlock() + m.triggers = append(m.triggers, &thresholdTrigger{ + triggerFunc: trigger, + }) +} + +func (m *memoryLimitController) checkTrigger(prevUsage int64, nextUsage int64) { + nextUsagePercent := float64(nextUsage) / float64(m.limit) + prevUsagePercent := float64(prevUsage) / float64(m.limit) + if nextUsagePercent >= m.triggerThreshold && prevUsagePercent < m.triggerThreshold { + for _, trigger := range m.triggers { + if trigger.canTryRunning() { + go func(trigger *thresholdTrigger) { + trigger.triggerFunc() + trigger.setRunning(false) + }(trigger) + } + } + } +} diff --git a/pulsar/internal/memory_limit_controller_test.go b/pulsar/internal/memory_limit_controller_test.go index a62c6e6dda..81623eca10 100644 --- a/pulsar/internal/memory_limit_controller_test.go +++ b/pulsar/internal/memory_limit_controller_test.go @@ -28,7 +28,7 @@ import ( func TestLimit(t *testing.T) { - mlc := NewMemoryLimitController(100) + mlc := NewMemoryLimitController(100, 1.0) for i := 0; i < 101; i++ { assert.True(t, mlc.TryReserveMemory(1)) @@ -57,7 +57,7 @@ func TestLimit(t *testing.T) { } func TestDisableLimit(t *testing.T) { - mlc := NewMemoryLimitController(-1) + mlc := NewMemoryLimitController(-1, 1.0) assert.True(t, mlc.TryReserveMemory(1000000)) assert.True(t, mlc.ReserveMemory(context.Background(), 1000000)) mlc.ReleaseMemory(1000000) @@ -65,7 +65,7 @@ func TestDisableLimit(t *testing.T) { } func TestMultiGoroutineTryReserveMem(t *testing.T) { - mlc := NewMemoryLimitController(10000) + mlc := NewMemoryLimitController(10000, 1.0) // Multi goroutine try reserve memory. wg := sync.WaitGroup{} @@ -87,7 +87,7 @@ func TestMultiGoroutineTryReserveMem(t *testing.T) { } func TestReserveWithContext(t *testing.T) { - mlc := NewMemoryLimitController(100) + mlc := NewMemoryLimitController(100, 1.0) assert.True(t, mlc.TryReserveMemory(101)) gorNum := 10 @@ -120,7 +120,7 @@ func TestReserveWithContext(t *testing.T) { } func TestBlocking(t *testing.T) { - mlc := NewMemoryLimitController(100) + mlc := NewMemoryLimitController(100, 1.0) assert.True(t, mlc.TryReserveMemory(101)) assert.Equal(t, int64(101), mlc.CurrentUsage()) assert.InDelta(t, 1.01, mlc.CurrentUsagePercent(), 0.000001) @@ -146,7 +146,7 @@ func TestBlocking(t *testing.T) { } func TestStepRelease(t *testing.T) { - mlc := NewMemoryLimitController(100) + mlc := NewMemoryLimitController(100, 1.0) assert.True(t, mlc.TryReserveMemory(101)) assert.Equal(t, int64(101), mlc.CurrentUsage()) assert.InDelta(t, 1.01, mlc.CurrentUsagePercent(), 0.000001) @@ -169,6 +169,57 @@ func TestStepRelease(t *testing.T) { assert.Equal(t, int64(101), mlc.CurrentUsage()) } +func TestRegisterTrigger(t *testing.T) { + mlc := NewMemoryLimitController(100, 0.95) + triggeredResult1 := false + triggeredResult2 := false + finishCh := make(chan struct{}, 2) + + mlc.RegisterTrigger(func() { + triggeredResult1 = true + finishCh <- struct{}{} + }) + + mlc.RegisterTrigger(func() { + triggeredResult2 = true + finishCh <- struct{}{} + }) + + mlc.TryReserveMemory(50) + timer := time.NewTimer(time.Millisecond * 500) + select { + case <-finishCh: + assert.Fail(t, "should not be triggered") + case <-timer.C: + } + + mlc.TryReserveMemory(45) + timer.Reset(time.Millisecond * 500) + for i := 0; i < 2; i++ { + select { + case <-finishCh: + case <-timer.C: + assert.Fail(t, "trigger timeout") + } + } + + assert.True(t, triggeredResult1) + assert.True(t, triggeredResult2) + + triggeredResult2 = false + mlc.ReleaseMemory(1) + mlc.ForceReserveMemory(1) + timer.Reset(time.Millisecond * 500) + for i := 0; i < 2; i++ { + select { + case <-finishCh: + case <-timer.C: + assert.Fail(t, "trigger timeout") + } + } + assert.True(t, triggeredResult2) +} + func reserveMemory(mlc MemoryLimitController, ch chan int) { mlc.ReserveMemory(context.Background(), 1) ch <- 1 From 09dea663222505d1c86f8f0ae24f95bc666f11b1 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Fri, 24 Mar 2023 17:13:04 +0800 Subject: [PATCH 040/348] [feat][txn]Implement transactionImpl (#984) Master Issue:/~https://github.com/apache/pulsar-client-go/issues/932 ### Motivation Implement transaction coordinator client. ### Modifications 1. Implement transaction coordinator 2. implement transactionImpl 3. Implement transaction in producer and consumer API --- pulsar/error.go | 28 ++- pulsar/transaction.go | 48 ++++- pulsar/transaction_coordinator_client.go | 35 +++- pulsar/transaction_impl.go | 237 +++++++++++++++++++++++ pulsar/transaction_test.go | 139 ++++++++++++- 5 files changed, 461 insertions(+), 26 deletions(-) create mode 100644 pulsar/transaction_impl.go diff --git a/pulsar/error.go b/pulsar/error.go index 0aa1e3c20d..73a0b6067b 100644 --- a/pulsar/error.go +++ b/pulsar/error.go @@ -17,7 +17,11 @@ package pulsar -import "fmt" +import ( + "fmt" + + proto "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" +) // Result used to represent pulsar processing is an alias of type int. type Result int @@ -103,14 +107,11 @@ const ( ProducerClosed // SchemaFailure means the payload could not be encoded using the Schema SchemaFailure - - // ReachMaxPendingOps means the pending operations in transaction_impl coordinator reach the maximum. - ReachMaxPendingOps // InvalidStatus means the component status is not as expected. InvalidStatus - // TransactionError means this is a transaction related error - TransactionError - + // TransactionNoFoundError The transaction is not exist in the transaction coordinator, It may be an error txn + // or already ended. + TransactionNoFoundError // ClientMemoryBufferIsFull client limit buffer is full ClientMemoryBufferIsFull ) @@ -221,7 +222,20 @@ func getResultStr(r Result) string { return "SchemaFailure" case ClientMemoryBufferIsFull: return "ClientMemoryBufferIsFull" + case TransactionNoFoundError: + return "TransactionNoFoundError" default: return fmt.Sprintf("Result(%d)", r) } } + +func getErrorFromServerError(serverError *proto.ServerError) error { + switch *serverError { + case proto.ServerError_TransactionNotFound: + return newError(TransactionNoFoundError, serverError.String()) + case proto.ServerError_InvalidTxnStatus: + return newError(InvalidStatus, serverError.String()) + default: + return newError(UnknownError, serverError.String()) + } +} diff --git a/pulsar/transaction.go b/pulsar/transaction.go index ae7c673046..60e1d2bf2b 100644 --- a/pulsar/transaction.go +++ b/pulsar/transaction.go @@ -17,7 +17,53 @@ package pulsar +import ( + "context" +) + +// TxnState The state of the transaction. Check the state of the transaction before executing some operation +// with the transaction is necessary. +type TxnState int32 + +const ( + _ TxnState = iota + // TxnOpen The transaction in TxnOpen state can be used to send/ack messages. + TxnOpen + // TxnCommitting The state of the transaction will be TxnCommitting after the commit method is called. + // The transaction in TxnCommitting state can be committed again. + TxnCommitting + // TxnAborting The state of the transaction will be TxnAborting after the abort method is called. + // The transaction in TxnAborting state can be aborted again. + TxnAborting + // TxnCommitted The state of the transaction will be TxnCommitted after the commit method is executed success. + // This means that all the operations with the transaction are success. + TxnCommitted + // TxnAborted The state of the transaction will be TxnAborted after the abort method is executed success. + // This means that all the operations with the transaction are aborted. + TxnAborted + // TxnError The state of the transaction will be TxnError after the operation of transaction get a non-retryable error. + TxnError + // TxnTimeout The state of the transaction will be TxnTimeout after the transaction timeout. + TxnTimeout +) + +// TxnID An identifier for representing a transaction. type TxnID struct { - mostSigBits uint64 + // mostSigBits The most significant 64 bits of this TxnID. + mostSigBits uint64 + // leastSigBits The least significant 64 bits of this TxnID. leastSigBits uint64 } + +// Transaction used to guarantee exactly-once +type Transaction interface { + //Commit You can commit the transaction after all the sending/acknowledging operations with the transaction success. + Commit(context.Context) error + //Abort You can abort the transaction when you want to abort all the sending/acknowledging operations + // with the transaction. + Abort(context.Context) error + //GetState Get the state of the transaction. + GetState() TxnState + //GetTxnID Get the identified ID of the transaction. + GetTxnID() TxnID +} diff --git a/pulsar/transaction_coordinator_client.go b/pulsar/transaction_coordinator_client.go index 82d1490398..1535fad113 100644 --- a/pulsar/transaction_coordinator_client.go +++ b/pulsar/transaction_coordinator_client.go @@ -114,14 +114,16 @@ func (tc *transactionCoordinatorClient) newTransaction(timeout time.Duration) (* TxnTtlSeconds: proto.Uint64(uint64(timeout.Milliseconds())), } - cnx, err := tc.client.rpcClient.RequestOnCnx(tc.cons[nextTcID], requestID, pb.BaseCommand_NEW_TXN, cmdNewTxn) + res, err := tc.client.rpcClient.RequestOnCnx(tc.cons[nextTcID], requestID, pb.BaseCommand_NEW_TXN, cmdNewTxn) tc.semaphore.Release() if err != nil { return nil, err + } else if res.Response.NewTxnResponse.Error != nil { + return nil, getErrorFromServerError(res.Response.NewTxnResponse.Error) } - return &TxnID{*cnx.Response.NewTxnResponse.TxnidMostBits, - *cnx.Response.NewTxnResponse.TxnidLeastBits}, nil + return &TxnID{*res.Response.NewTxnResponse.TxnidMostBits, + *res.Response.NewTxnResponse.TxnidLeastBits}, nil } // addPublishPartitionToTxn register the partitions which published messages with the transactionImpl. @@ -137,10 +139,15 @@ func (tc *transactionCoordinatorClient) addPublishPartitionToTxn(id *TxnID, part TxnidLeastBits: proto.Uint64(id.leastSigBits), Partitions: partitions, } - _, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, + res, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, pb.BaseCommand_ADD_PARTITION_TO_TXN, cmdAddPartitions) tc.semaphore.Release() - return err + if err != nil { + return err + } else if res.Response.AddPartitionToTxnResponse.Error != nil { + return getErrorFromServerError(res.Response.AddPartitionToTxnResponse.Error) + } + return nil } // addSubscriptionToTxn register the subscription which acked messages with the transactionImpl. @@ -160,10 +167,15 @@ func (tc *transactionCoordinatorClient) addSubscriptionToTxn(id *TxnID, topic st TxnidLeastBits: proto.Uint64(id.leastSigBits), Subscription: []*pb.Subscription{sub}, } - _, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, + res, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, pb.BaseCommand_ADD_SUBSCRIPTION_TO_TXN, cmdAddSubscription) tc.semaphore.Release() - return err + if err != nil { + return err + } else if res.Response.AddSubscriptionToTxnResponse.Error != nil { + return getErrorFromServerError(res.Response.AddSubscriptionToTxnResponse.Error) + } + return nil } // endTxn commit or abort the transactionImpl. @@ -178,9 +190,14 @@ func (tc *transactionCoordinatorClient) endTxn(id *TxnID, action pb.TxnAction) e TxnidMostBits: proto.Uint64(id.mostSigBits), TxnidLeastBits: proto.Uint64(id.leastSigBits), } - _, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, pb.BaseCommand_END_TXN, cmdEndTxn) + res, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, pb.BaseCommand_END_TXN, cmdEndTxn) tc.semaphore.Release() - return err + if err != nil { + return err + } else if res.Response.EndTxnResponse.Error != nil { + return getErrorFromServerError(res.Response.EndTxnResponse.Error) + } + return nil } func getTCAssignTopicName(partition uint64) string { diff --git a/pulsar/transaction_impl.go b/pulsar/transaction_impl.go new file mode 100644 index 0000000000..7cc93eca0a --- /dev/null +++ b/pulsar/transaction_impl.go @@ -0,0 +1,237 @@ +// 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 pulsar + +import ( + "context" + "sync" + "sync/atomic" + "time" + + pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" + "github.com/apache/pulsar-client-go/pulsar/log" +) + +type subscription struct { + topic string + subscription string +} + +type transaction struct { + sync.Mutex + txnID TxnID + state TxnState + tcClient *transactionCoordinatorClient + registerPartitions map[string]bool + registerAckSubscriptions map[subscription]bool + // opsFlow It has two effects: + // 1. Wait all the operations of sending and acking messages with the transaction complete + // by reading msg from the chan. + // 2. Prevent sending or acking messages with a committed or aborted transaction. + // opsCount is record the number of the uncompleted operations. + // opsFlow + // Write: + // 1. When the transaction is created, a bool will be written to opsFlow chan. + // 2. When the opsCount decrement from 1 to 0, a new bool will be written to opsFlow chan. + // 3. When get a retryable error after committing or aborting the transaction, + // a bool will be written to opsFlow chan. + // Read: + // 1. When the transaction is committed or aborted, a bool will be read from opsFlow chan. + // 2. When the opsCount increment from 0 to 1, a bool will be read from opsFlow chan. + opsFlow chan bool + opsCount int32 + opTimeout time.Duration + log log.Logger +} + +func newTransaction(id TxnID, tcClient *transactionCoordinatorClient, timeout time.Duration) *transaction { + transaction := &transaction{ + txnID: id, + state: TxnOpen, + registerPartitions: make(map[string]bool), + registerAckSubscriptions: make(map[subscription]bool), + opsFlow: make(chan bool, 1), + opTimeout: 5 * time.Second, + tcClient: tcClient, + } + //This means there are not pending requests with this transaction. The transaction can be committed or aborted. + transaction.opsFlow <- true + go func() { + //Set the state of the transaction to timeout after timeout + <-time.After(timeout) + atomic.CompareAndSwapInt32((*int32)(&transaction.state), int32(TxnOpen), int32(TxnTimeout)) + }() + transaction.log = tcClient.log.SubLogger(log.Fields{}) + return transaction +} + +func (txn *transaction) GetState() TxnState { + return txn.state +} + +func (txn *transaction) Commit(ctx context.Context) error { + if !(atomic.CompareAndSwapInt32((*int32)(&txn.state), int32(TxnOpen), int32(TxnCommitting)) || + txn.state == TxnCommitting) { + return newError(InvalidStatus, "Expect transaction state is TxnOpen but "+txn.state.string()) + } + + //Wait for all operations to complete + select { + case <-txn.opsFlow: + case <-time.After(txn.opTimeout): + return newError(TimeoutError, "There are some operations that are not completed after the timeout.") + } + //Send commit transaction command to transaction coordinator + err := txn.tcClient.endTxn(&txn.txnID, pb.TxnAction_COMMIT) + if err == nil { + atomic.StoreInt32((*int32)(&txn.state), int32(TxnCommitted)) + } else { + if err.(*Error).Result() == TransactionNoFoundError || err.(*Error).Result() == InvalidStatus { + atomic.StoreInt32((*int32)(&txn.state), int32(TxnError)) + return err + } + txn.opsFlow <- true + } + return err +} + +func (txn *transaction) Abort(ctx context.Context) error { + if !(atomic.CompareAndSwapInt32((*int32)(&txn.state), int32(TxnOpen), int32(TxnAborting)) || + txn.state == TxnAborting) { + return newError(InvalidStatus, "Expect transaction state is TxnOpen but "+txn.state.string()) + } + + //Wait for all operations to complete + select { + case <-txn.opsFlow: + case <-time.After(txn.opTimeout): + return newError(TimeoutError, "There are some operations that are not completed after the timeout.") + } + //Send abort transaction command to transaction coordinator + err := txn.tcClient.endTxn(&txn.txnID, pb.TxnAction_ABORT) + if err == nil { + atomic.StoreInt32((*int32)(&txn.state), int32(TxnAborted)) + } else { + if err.(*Error).Result() == TransactionNoFoundError || err.(*Error).Result() == InvalidStatus { + atomic.StoreInt32((*int32)(&txn.state), int32(TxnError)) + } else { + txn.opsFlow <- true + } + } + return err +} + +func (txn *transaction) registerSendOrAckOp() error { + if atomic.AddInt32(&txn.opsCount, 1) == 1 { + //There are new operations that not completed + select { + case <-txn.opsFlow: + return nil + case <-time.After(txn.opTimeout): + if _, err := txn.checkIfOpen(); err != nil { + return err + } + return newError(TimeoutError, "Failed to get the semaphore to register the send/ack operation") + } + } + return nil +} + +func (txn *transaction) endSendOrAckOp(err error) { + if err != nil { + atomic.StoreInt32((*int32)(&txn.state), int32(TxnError)) + } + if atomic.AddInt32(&txn.opsCount, -1) == 0 { + //This means there are not pending send/ack requests + txn.opsFlow <- true + } +} + +func (txn *transaction) registerProducerTopic(topic string) error { + isOpen, err := txn.checkIfOpen() + if !isOpen { + return err + } + _, ok := txn.registerPartitions[topic] + if !ok { + txn.Lock() + defer txn.Unlock() + if _, ok = txn.registerPartitions[topic]; !ok { + err := txn.tcClient.addPublishPartitionToTxn(&txn.txnID, []string{topic}) + if err != nil { + return err + } + txn.registerPartitions[topic] = true + } + } + return nil +} + +func (txn *transaction) registerAckTopic(topic string, subName string) error { + isOpen, err := txn.checkIfOpen() + if !isOpen { + return err + } + sub := subscription{ + topic: topic, + subscription: subName, + } + _, ok := txn.registerAckSubscriptions[sub] + if !ok { + txn.Lock() + defer txn.Unlock() + if _, ok = txn.registerAckSubscriptions[sub]; !ok { + err := txn.tcClient.addSubscriptionToTxn(&txn.txnID, topic, subName) + if err != nil { + return err + } + txn.registerAckSubscriptions[sub] = true + } + } + return nil +} + +func (txn *transaction) GetTxnID() TxnID { + return txn.txnID +} + +func (txn *transaction) checkIfOpen() (bool, error) { + if txn.state == TxnOpen { + return true, nil + } + return false, newError(InvalidStatus, "Expect transaction state is TxnOpen but "+txn.state.string()) +} + +func (state TxnState) string() string { + switch state { + case TxnOpen: + return "TxnOpen" + case TxnCommitting: + return "TxnCommitting" + case TxnAborting: + return "TxnAborting" + case TxnCommitted: + return "TxnCommitted" + case TxnAborted: + return "TxnAborted" + case TxnTimeout: + return "TxnTimeout" + default: + return "Unknown" + } +} diff --git a/pulsar/transaction_test.go b/pulsar/transaction_test.go index 14a652fdee..362e4d2eb6 100644 --- a/pulsar/transaction_test.go +++ b/pulsar/transaction_test.go @@ -18,11 +18,15 @@ package pulsar import ( - pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" - "github.com/stretchr/testify/assert" - + "context" + "errors" + "fmt" "testing" "time" + + pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestTCClient(t *testing.T) { @@ -79,6 +83,126 @@ func TestTCClient(t *testing.T) { defer client.Close() } +//Test points: +// 1. Abort and commit txn. +// 1. Do nothing, just open a transaction and then commit it or abort it. +// The operations of committing/aborting txn should success at the first time and fail at the second time. +// 2. The internal API, registerSendOrAckOp and endSendOrAckOp +// 1. Register 4 operation but only end 3 operations, the transaction can not be committed or aborted. +// 2. Register 3 operation and end 3 operation the transaction can be committed and aborted. +// 3. Register an operation and end the operation with an error, +// and then the state of the transaction should be replaced to errored. +// 3. The internal API, registerAckTopic and registerProducerTopic +// 1. Register ack topic and send topic, and call http request to get the stats of the transaction +// to do verification. + +// TestTxnImplCommitOrAbort Test abort and commit txn +func TestTxnImplCommitOrAbort(t *testing.T) { + tc, _ := createTcClient(t) + //1. Open a transaction and then commit it. + //The operations of committing txn1 should success at the first time and fail at the second time. + txn1 := createTxn(tc, t) + err := txn1.Commit(context.Background()) + require.Nil(t, err, fmt.Sprintf("Failed to commit the transaction %d:%d\n", txn1.txnID.mostSigBits, + txn1.txnID.leastSigBits)) + txn1.state = TxnOpen + txn1.opsFlow <- true + err = txn1.Commit(context.Background()) + assert.Equal(t, err.(*Error).Result(), TransactionNoFoundError) + assert.Equal(t, txn1.GetState(), TxnError) + //2. Open a transaction and then abort it. + //The operations of aborting txn2 should success at the first time and fail at the second time. + id2, err := tc.newTransaction(time.Hour) + require.Nil(t, err, "Failed to new a transaction") + txn2 := newTransaction(*id2, tc, time.Hour) + err = txn2.Abort(context.Background()) + require.Nil(t, err, fmt.Sprintf("Failed to abort the transaction %d:%d\n", + id2.mostSigBits, id2.leastSigBits)) + txn2.state = TxnOpen + txn2.opsFlow <- true + err = txn2.Abort(context.Background()) + assert.Equal(t, err.(*Error).Result(), TransactionNoFoundError) + assert.Equal(t, txn1.GetState(), TxnError) + err = txn2.registerSendOrAckOp() + assert.Equal(t, err.(*Error).Result(), InvalidStatus) + err = txn1.registerSendOrAckOp() + assert.Equal(t, err.(*Error).Result(), InvalidStatus) +} + +// TestRegisterOpAndEndOp Test the internal API including the registerSendOrAckOp and endSendOrAckOp. +func TestRegisterOpAndEndOp(t *testing.T) { + tc, _ := createTcClient(t) + //1. Register 4 operation but only end 3 operations, the transaction can not be committed or aborted. + res := registerOpAndEndOp(t, tc, 4, 3, nil, true) + assert.Equal(t, res.(*Error).Result(), TimeoutError) + res = registerOpAndEndOp(t, tc, 4, 3, nil, false) + assert.Equal(t, res.(*Error).Result(), TimeoutError) + + //2. Register 3 operation and end 3 operation the transaction can be committed and aborted. + res = registerOpAndEndOp(t, tc, 3, 3, nil, true) + assert.Nil(t, res) + res = registerOpAndEndOp(t, tc, 3, 3, nil, false) + assert.Nil(t, res) + //3. Register an operation and end the operation with an error, + // and then the state of the transaction should be replaced to errored. + res = registerOpAndEndOp(t, tc, 4, 4, errors.New(""), true) + assert.Equal(t, res.(*Error).Result(), InvalidStatus) + res = registerOpAndEndOp(t, tc, 4, 4, errors.New(""), false) + assert.Equal(t, res.(*Error).Result(), InvalidStatus) +} + +// TestRegisterTopic Test the internal API, registerAckTopic and registerProducerTopic +func TestRegisterTopic(t *testing.T) { + //1. Prepare: create PulsarClient and init transaction coordinator client. + topic := newTopicName() + sub := "my-sub" + tc, client := createTcClient(t) + //2. Prepare: create Topic and Subscription. + _, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: sub, + }) + assert.NoError(t, err) + txn := createTxn(tc, t) + //3. Create a topic and register topic and subscription. + err = txn.registerAckTopic(topic, sub) + require.Nil(t, err, "Failed to register ack topic.") + err = txn.registerProducerTopic(topic) + require.Nil(t, err, "Failed to register ack topic.") + //4. Call http request to get the stats of the transaction to do verification. + stats2, err := transactionStats(&txn.txnID) + assert.NoError(t, err) + topics := stats2["producedPartitions"].(map[string]interface{}) + subTopics := stats2["ackedPartitions"].(map[string]interface{}) + assert.NotNil(t, topics[topic]) + assert.NotNil(t, subTopics[topic]) + subs := subTopics[topic].(map[string]interface{}) + assert.NotNil(t, subs[sub]) +} + +func registerOpAndEndOp(t *testing.T, tc *transactionCoordinatorClient, rp int, ep int, err error, commit bool) error { + txn := createTxn(tc, t) + for i := 0; i < rp; i++ { + err := txn.registerSendOrAckOp() + assert.Nil(t, err) + } + for i := 0; i < ep; i++ { + txn.endSendOrAckOp(err) + } + if commit { + err = txn.Commit(context.Background()) + } else { + err = txn.Abort(context.Background()) + } + return err +} + +func createTxn(tc *transactionCoordinatorClient, t *testing.T) *transaction { + id, err := tc.newTransaction(time.Hour) + require.Nil(t, err, "Failed to new a transaction.") + return newTransaction(*id, tc, time.Hour) +} + // createTcClient Create a transaction coordinator client to send request func createTcClient(t *testing.T) (*transactionCoordinatorClient, *client) { c, err := NewClient(ClientOptions{ @@ -86,13 +210,10 @@ func createTcClient(t *testing.T) (*transactionCoordinatorClient, *client) { TLSTrustCertsFilePath: caCertsPath, Authentication: NewAuthenticationTLS(tlsClientCertPath, tlsClientKeyPath), }) - if err != nil { - t.Fatalf("Failed to create client due to %s", err.Error()) - } + require.Nil(t, err, "Failed to create client.") tcClient := newTransactionCoordinatorClientImpl(c.(*client)) - if err = tcClient.start(); err != nil { - t.Fatalf("Failed to start transaction coordinator due to %s", err.Error()) - } + err = tcClient.start() + require.Nil(t, err, "Failed to start transaction coordinator.") return tcClient, c.(*client) } From 732f3d093e728c67f0a34382666f0daef25efdca Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Mon, 3 Apr 2023 13:48:53 +0800 Subject: [PATCH 041/348] Add 0.10.0 change log (#997) * Add 0.10.0 change log * Apply suggestions from code review Co-authored-by: Jun Ma <60642177+momo-jun@users.noreply.github.com> * Refine changelog --------- Co-authored-by: Jun Ma <60642177+momo-jun@users.noreply.github.com> --- CHANGELOG.md | 86 ++++++++++++++++++++++++++++++++++++++++++++++++++++ VERSION | 2 +- stable.txt | 2 +- 3 files changed, 88 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e68f59b6c0..b3e9351a45 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,6 +23,92 @@ All notable changes to this project will be documented in this file. +[0.10.0] 2023-03-27 + +## Feature +* Support chunking for big messages by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/805 +* Add BackoffPolicy to `reader` and improve test case by @labuladong in /~https://github.com/apache/pulsar-client-go/pull/889 +* Support cumulative acknowledgment by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/903 +* Support consumer event listener by @labuladong in /~https://github.com/apache/pulsar-client-go/pull/904 +* Allow CustomProperties when sending messages for retry by @ngoyal16 in /~https://github.com/apache/pulsar-client-go/pull/916 +* Support batch index ACK by @BewareMyPower in /~https://github.com/apache/pulsar-client-go/pull/938 +* Support Exclusive Producer access mode by @shibd in /~https://github.com/apache/pulsar-client-go/pull/944 +* Add transactionCoordinatorClient by @liangyepianzhou in /~https://github.com/apache/pulsar-client-go/pull/953 +* Support memory limit for the producer by @shibd in /~https://github.com/apache/pulsar-client-go/pull/955 +* Support grouping ACK requests by time and size by @BewareMyPower in /~https://github.com/apache/pulsar-client-go/pull/957 +* Support WaitForExclusive producer access mode by @shibd in /~https://github.com/apache/pulsar-client-go/pull/958 +* Support Copper Argos in the Athenz auth provider by @massakam in /~https://github.com/apache/pulsar-client-go/pull/960 +* Support auto-release idle connections by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/963 +* Support batch index ACK and set max number of messages in batch for the perf tool by @BewareMyPower in /~https://github.com/apache/pulsar-client-go/pull/967 +* Support auto-scaled consumer receiver queue by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/976 +* Implement transactionImpl by @liangyepianzhou in /~https://github.com/apache/pulsar-client-go/pull/984 +* Expose the chunk config of the consumer to the reader by @CrazyCollin in /~https://github.com/apache/pulsar-client-go/pull/987 +* Support consumer client memory limit by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/991 + + +## Improve +* Nack the message in dlqrouter when sending errors by @leizhiyuan in /~https://github.com/apache/pulsar-client-go/pull/592 +* Fix TLS certificates that do not include IP SANS, save hostname before switching to a physical address by @dinghram in /~https://github.com/apache/pulsar-client-go/pull/812 +* Fix the availablePermits leak that could cause the consumer stuck by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/835 +* Read module version info from golang runtime by @pgier in /~https://github.com/apache/pulsar-client-go/pull/856 +* Fix typo in `consumer.go` by @sekfung in /~https://github.com/apache/pulsar-client-go/pull/857 +* Fix marshalling `time.Time{}` to `uint64` by @aymkhalil in /~https://github.com/apache/pulsar-client-go/pull/865 +* Use the `DATA` constant as the prefix in OAuth2 KeyFileProvider by @Niennienzz in /~https://github.com/apache/pulsar-client-go/pull/866 +* Fix bot cannot get the pr link by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/868 +* Fix PR template by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/869 +* Add go test flag '-v' for more clearly CI log by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/871 +* Fix the dispatcher() stuck caused by availablePermitsCh by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/875 +* Fix the Send() stuck caused by callback() not being called by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/880 +* Fix the data race of ackReq.err by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/881 +* Add data URL format to read the key file by @nodece in /~https://github.com/apache/pulsar-client-go/pull/883 +* Prevent consumer panic on de-serializing message if schema not found by @GPrabhudas in /~https://github.com/apache/pulsar-client-go/pull/886 +* Fix the conditions of loading TLS certificates by @nodece in /~https://github.com/apache/pulsar-client-go/pull/888 +* Fix default retry and dlq topic name as per the doc by @ngoyal16 in /~https://github.com/apache/pulsar-client-go/pull/891 +* Add NewMessageID() method by @crossoverJie in /~https://github.com/apache/pulsar-client-go/pull/893 +* Use protocolbuffers instead of gogo by @nodece in /~https://github.com/apache/pulsar-client-go/pull/895 +* Fix the compression broken when batching is disabled by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/902 +* Add messageId and topic as props of DLQ message by @GPrabhudas in /~https://github.com/apache/pulsar-client-go/pull/907 +* Update go version to 1.18 by @pgier in /~https://github.com/apache/pulsar-client-go/pull/911 +* Move out the auth package from internal by @nodece in /~https://github.com/apache/pulsar-client-go/pull/914 +* Remove the `clearMessageQueuesCh` in `partitionConsumer.dispatcher()` by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/921 +* Remove the outdated interface description of `SeekByTime` by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/924 +* Handle nil value message correctly in table-view by @Demogorgon314 in /~https://github.com/apache/pulsar-client-go/pull/930 +* Migrate from the deprecated io/ioutil package by @reugn in /~https://github.com/apache/pulsar-client-go/pull/942 +* Update the Cobra library to significantly reduce the dependency tree by @reugn in /~https://github.com/apache/pulsar-client-go/pull/943 +* Remove go1.11 code leftovers by @reugn in /~https://github.com/apache/pulsar-client-go/pull/946 +* Use pkg.go.dev badge in the readme by @reugn in /~https://github.com/apache/pulsar-client-go/pull/947 +* Improve test script by @nodece in /~https://github.com/apache/pulsar-client-go/pull/951 +* Optimize the performance by passing MessageID implementations by pointers by @BewareMyPower in /~https://github.com/apache/pulsar-client-go/pull/968 +* Fix flaky Key_Shared subscription-related tests by @BewareMyPower in /~https://github.com/apache/pulsar-client-go/pull/970 +* Refactor the toTrackingMessageID() by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/972 +* Prevent RPC client panic on RPC response if `ProducerReady` is nil by @sekfung in /~https://github.com/apache/pulsar-client-go/pull/973 +* Fix nack backoff policy logic by @wolfstudy in /~https://github.com/apache/pulsar-client-go/pull/974 +* Fix license information for go-rate by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/975 +* Fix the data race in checkAndCleanIdleConnections by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/981 +* Setup rate limiter for TestChunksEnqueueFailed to reduce flaky by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/982 +* Fix the message is blocked on the AckGroupingTracker.isDuplicate method by @shibd in /~https://github.com/apache/pulsar-client-go/pull/986 +* Optimize batch index ACK performance by @BewareMyPower in /~https://github.com/apache/pulsar-client-go/pull/988 +* Add more precise producer rate limiter by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/989 +* Fix batched messages not ACKed correctly when batch index ACK is disabled by @BewareMyPower in /~https://github.com/apache/pulsar-client-go/pull/994 +* Fix panic caused by retryAssert() by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/996 + +## New Contributors +* @sekfung made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/857 +* @Gleiphir2769 made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/835 +* @michaeljmarshall made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/861 +* @aymkhalil made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/865 +* @RobertIndie made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/868 +* @dinghram made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/812 +* @labuladong made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/889 +* @Niennienzz made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/866 +* @crossoverJie made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/893 +* @ngoyal16 made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/891 +* @Demogorgon314 made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/930 +* @shibd made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/944 +* @liangyepianzhou made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/953 +* @tisonkun made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/975 +* @CrazyCollin made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/987 + [0.9.0] 2022-07-07 ## Feature diff --git a/VERSION b/VERSION index f6fadb296b..46cd5a5b14 100644 --- a/VERSION +++ b/VERSION @@ -1,3 +1,3 @@ // This version number refers to the currently released version number // Please fix the version when release. -v0.9.0 +v0.10.0 diff --git a/stable.txt b/stable.txt index d9c4fc335b..fab6099c07 100644 --- a/stable.txt +++ b/stable.txt @@ -1,3 +1,3 @@ // This version number refers to the current stable version, generally is `VERSION - 1`. // Please fix the version when release. -v0.9.0 +v0.10.0 From 4836eacf3319866b00b93fc634d96a785f6e7170 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 5 Apr 2023 10:39:37 +0800 Subject: [PATCH 042/348] [Doc] Fix missing link in the release process (#1000) --- docs/release-process.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/release-process.md b/docs/release-process.md index d2d54454b4..aa79f9ea34 100644 --- a/docs/release-process.md +++ b/docs/release-process.md @@ -13,7 +13,7 @@ In general, you need to perform the following steps: 9. Announce the release. ### Requirements -- [GPG keys to sign release artifacts](/~https://github.com/apache/pulsar/wiki/Create-GPG-keys-to-sign-release-artifacts) +- [Creating GPG keys to sign release artifacts](https://pulsar.apache.org/contribute/create-gpg-keys/) ## Steps in detail From 16b1fa34339843dbea5b9e57c7b2e51d3ded11a0 Mon Sep 17 00:00:00 2001 From: tison Date: Mon, 10 Apr 2023 23:34:43 +0800 Subject: [PATCH 043/348] ci: stablize golangci-lint task (#1007) Modifications 1. Replace the usage of golangci-lint action with barely bin commands. 2. Support run with the specified version with one command `make lint` and store the binutil project-wise. Signed-off-by: tison --- .github/workflows/bot.yml | 37 ++++++++++++++++---------- .github/workflows/ci.yml | 47 ++++++++++++++++++++++++++++++++++ .github/workflows/project.yml | 39 ---------------------------- .golangci.yml | 22 +++++++++++++--- Makefile | 11 ++++++-- pulsar/internal/http_client.go | 1 + 6 files changed, 99 insertions(+), 58 deletions(-) create mode 100644 .github/workflows/ci.yml delete mode 100644 .github/workflows/project.yml diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index eb86c56570..343963efde 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -1,24 +1,33 @@ -name: Bot tests +# 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. + +name: Pulsar Bot on: issue_comment: types: [created] - pull_request: - types: [closed] jobs: - bot: - runs-on: ubuntu-latest - timeout-minutes: 120 - + pulsarbot: + runs-on: ubuntu-20.04 + timeout-minutes: 10 + if: github.event_name == 'issue_comment' && contains(github.event.comment.body, '/pulsarbot') steps: - - name: checkout - uses: actions/checkout@v2 - with: - fetch-depth: 100 - ref: ${{ github.event.pull_request.head.sha }} - name: Execute pulsarbot command - id: pulsarbot - if: github.event_name == 'issue_comment' && startsWith(github.event.comment.body, '/pulsarbot') + id: pulsarbot env: GITHUB_TOKEN: ${{ secrets.GO_CLIENT_BOT_TOKEN }} BOT_TARGET_REPOSITORY: apache/pulsar-client-go diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml new file mode 100644 index 0000000000..7c946b7652 --- /dev/null +++ b/.github/workflows/ci.yml @@ -0,0 +1,47 @@ +# 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. + +name: CI +on: [pull_request] +jobs: + build: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - uses: actions/setup-go@v3 + - run: make build + + lint: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - run: make lint + + integration-tests: + runs-on: ubuntu-latest + strategy: + matrix: + go-version: [1.16, 1.17, 1.18, 1.19] + steps: + - uses: actions/checkout@v3 + - name: clean docker cache + run: docker rmi $(docker images -q) -f && df -h + - uses: actions/setup-go@v3 + with: + go-version: ${{ matrix.go-version }} + - name: Run Tests + run: make test GO_VERSION=${{ matrix.go-version }} diff --git a/.github/workflows/project.yml b/.github/workflows/project.yml deleted file mode 100644 index a52a4be4f6..0000000000 --- a/.github/workflows/project.yml +++ /dev/null @@ -1,39 +0,0 @@ -name: CI -on: [pull_request] -jobs: - build: - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v3 - - uses: actions/setup-go@v3 - - name: build - run: make build - - lint: - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v3 - - uses: golangci/golangci-lint-action@v3 - with: - version: v1.50.1 - - integration-tests: - runs-on: ubuntu-latest - strategy: - matrix: - go-version: [1.16, 1.17, 1.18, 1.19] - steps: - - name: clean docker cache - run: | - docker rmi $(docker images -q) -f - df -h - - - uses: actions/setup-go@v3 - with: - go-version: ${{ matrix.go-version }} - - - name: Check out code into the Go module directory - uses: actions/checkout@v3 - - - name: Run Tests - run: make test GO_VERSION=${{ matrix.go-version }} diff --git a/.golangci.yml b/.golangci.yml index 2dc120a20f..1f1b42c5b5 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -1,6 +1,22 @@ -# Make sure golangci-lint is installed. -# /~https://github.com/golangci/golangci-lint#install -# We can execute `golangci-lint run` for code checking. +# 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. + +# Run `make lint` from the root path of this project to check code with golangci-lint. + run: deadline: 6m diff --git a/Makefile b/Makefile index 231c05eb55..433db8cb77 100644 --- a/Makefile +++ b/Makefile @@ -23,12 +23,19 @@ PULSAR_IMAGE = apachepulsar/pulsar:$(PULSAR_VERSION) GO_VERSION ?= 1.18 GOLANG_IMAGE = golang:$(GO_VERSION) +# Golang standard bin directory. +GOPATH ?= $(shell go env GOPATH) +GOROOT ?= $(shell go env GOROOT) + build: go build ./pulsar go build -o bin/pulsar-perf ./perf -lint: - golangci-lint run +lint: bin/golangci-lint + bin/golangci-lint run + +bin/golangci-lint: + GOBIN=$(shell pwd)/bin go install github.com/golangci/golangci-lint/cmd/golangci-lint@v1.51.2 container: docker build -t ${IMAGE_NAME} --build-arg GOLANG_IMAGE="${GOLANG_IMAGE}" \ diff --git a/pulsar/internal/http_client.go b/pulsar/internal/http_client.go index dccc1431b3..c3b38f26ff 100644 --- a/pulsar/internal/http_client.go +++ b/pulsar/internal/http_client.go @@ -173,6 +173,7 @@ func (c *httpClient) GetWithQueryParams(endpoint string, obj interface{}, params return c.GetWithOptions(endpoint, obj, params, decode, nil) } +//nolint:bodyclose // false positive func (c *httpClient) GetWithOptions(endpoint string, obj interface{}, params map[string]string, decode bool, file io.Writer) ([]byte, error) { From 2154d70c0fd9c215b9e494046e094f665396c983 Mon Sep 17 00:00:00 2001 From: xiaolong ran Date: Fri, 14 Apr 2023 15:52:40 +0800 Subject: [PATCH 044/348] Fix reconnection backoff logic (#1008) Signed-off-by: xiaolongran --- pulsar/consumer_partition.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index fb77d0dc6a..a8ba7b5540 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -1525,6 +1525,11 @@ func (pc *partitionConsumer) reconnectToBroker() { maxRetry = int(*pc.options.maxReconnectToBroker) } + var ( + delayReconnectTime time.Duration + defaultBackoff = internal.DefaultBackoff{} + ) + for maxRetry != 0 { if pc.getConsumerState() != consumerReady { // Consumer is already closing @@ -1532,11 +1537,6 @@ func (pc *partitionConsumer) reconnectToBroker() { return } - var ( - delayReconnectTime time.Duration - defaultBackoff = internal.DefaultBackoff{} - ) - if pc.options.backoffPolicy == nil { delayReconnectTime = defaultBackoff.Next() } else { From 55f0c7cfda647b5ed0c19882790e92418b8bd651 Mon Sep 17 00:00:00 2001 From: labuladong Date: Fri, 14 Apr 2023 17:17:37 +0800 Subject: [PATCH 045/348] [improve][ci] try to fix bot workflow (#910) --- .github/workflows/bot.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index 343963efde..4641eb9912 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -29,6 +29,6 @@ jobs: - name: Execute pulsarbot command id: pulsarbot env: - GITHUB_TOKEN: ${{ secrets.GO_CLIENT_BOT_TOKEN }} + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} BOT_TARGET_REPOSITORY: apache/pulsar-client-go uses: apache/pulsar-test-infra/pulsarbot@master From fc4f2892a2f4af2c13485adf77ae6fc31a65910f Mon Sep 17 00:00:00 2001 From: ran Date: Sun, 23 Apr 2023 23:49:53 +0800 Subject: [PATCH 046/348] [feat][schema] Support the schema type ProtoNativeSchema (#1006) * Support the ProtoNativeSchema * fix * fix * fix * fix * fix style * fix license header * address comments * fix * fix * Fix typo in logs --------- Co-authored-by: Yunze Xu --- go.mod | 2 +- go.sum | 2 + integration-tests/pb/build.sh | 7 +- integration-tests/pb/hello.pb.go | 142 ++++++++++++++++++++++---- integration-tests/pb/hello.proto | 12 ++- integration-tests/pb/hi.pb.go | 168 +++++++++++++++++++++++++++++++ integration-tests/pb/hi.proto | 9 ++ pulsar/schema.go | 84 ++++++++++++++++ pulsar/schema_test.go | 65 +++++++++++- 9 files changed, 469 insertions(+), 22 deletions(-) create mode 100644 integration-tests/pb/hi.pb.go create mode 100644 integration-tests/pb/hi.proto diff --git a/go.mod b/go.mod index 3e7299b07b..69862b2080 100644 --- a/go.mod +++ b/go.mod @@ -27,7 +27,7 @@ require ( golang.org/x/mod v0.5.1 golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 golang.org/x/time v0.0.0-20191024005414-555d28b269f0 - google.golang.org/protobuf v1.26.0 + google.golang.org/protobuf v1.30.0 // indirect ) require ( diff --git a/go.sum b/go.sum index 528c45882f..3a3987443d 100644 --- a/go.sum +++ b/go.sum @@ -553,6 +553,8 @@ google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlba google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0 h1:bxAC2xTBsZGibn2RTntX0oH50xLsqy1OxA9tTL3p/lk= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= +google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/integration-tests/pb/build.sh b/integration-tests/pb/build.sh index 32c0b08a59..e1924581d0 100755 --- a/integration-tests/pb/build.sh +++ b/integration-tests/pb/build.sh @@ -18,4 +18,9 @@ # under the License. # -protoc --go_out=. --go_opt=paths=source_relative hello.proto +PROJECT_DIR=$(git rev-parse --show-toplevel) +SOURCE_PATH="$PROJECT_DIR/integration-tests/pb" +DEST_PATH="$PROJECT_DIR/integration-tests/pb" +echo "source_path: $SOURCE_PATH" +echo "dest_path: $DEST_PATH" +protoc -I=$SOURCE_PATH --go_out=$DEST_PATH $SOURCE_PATH/*.proto diff --git a/integration-tests/pb/hello.pb.go b/integration-tests/pb/hello.pb.go index 7884dfa3c4..541172c8bf 100644 --- a/integration-tests/pb/hello.pb.go +++ b/integration-tests/pb/hello.pb.go @@ -18,11 +18,11 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.28.1 -// protoc v3.21.9 +// protoc-gen-go v1.30.0 +// protoc v3.21.12 // source: hello.proto -package pb +package __ import ( protoreflect "google.golang.org/protobuf/reflect/protoreflect" @@ -43,8 +43,10 @@ type Test struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Num int32 `protobuf:"varint,1,opt,name=num,proto3" json:"num,omitempty"` - Msf string `protobuf:"bytes,2,opt,name=msf,proto3" json:"msf,omitempty"` + Num int32 `protobuf:"varint,1,opt,name=num,proto3" json:"num,omitempty"` + Msf string `protobuf:"bytes,2,opt,name=msf,proto3" json:"msf,omitempty"` + Foo *Foo `protobuf:"bytes,3,opt,name=foo,proto3" json:"foo,omitempty"` + HiContent *HiContent `protobuf:"bytes,4,opt,name=hiContent,proto3" json:"hiContent,omitempty"` } func (x *Test) Reset() { @@ -93,15 +95,102 @@ func (x *Test) GetMsf() string { return "" } +func (x *Test) GetFoo() *Foo { + if x != nil { + return x.Foo + } + return nil +} + +func (x *Test) GetHiContent() *HiContent { + if x != nil { + return x.HiContent + } + return nil +} + +type Foo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Value int32 `protobuf:"varint,2,opt,name=value,proto3" json:"value,omitempty"` + Names []string `protobuf:"bytes,3,rep,name=names,proto3" json:"names,omitempty"` +} + +func (x *Foo) Reset() { + *x = Foo{} + if protoimpl.UnsafeEnabled { + mi := &file_hello_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Foo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Foo) ProtoMessage() {} + +func (x *Foo) ProtoReflect() protoreflect.Message { + mi := &file_hello_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Foo.ProtoReflect.Descriptor instead. +func (*Foo) Descriptor() ([]byte, []int) { + return file_hello_proto_rawDescGZIP(), []int{1} +} + +func (x *Foo) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Foo) GetValue() int32 { + if x != nil { + return x.Value + } + return 0 +} + +func (x *Foo) GetNames() []string { + if x != nil { + return x.Names + } + return nil +} + var File_hello_proto protoreflect.FileDescriptor var file_hello_proto_rawDesc = []byte{ 0x0a, 0x0b, 0x68, 0x65, 0x6c, 0x6c, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x09, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x74, 0x65, 0x73, 0x74, 0x22, 0x2a, 0x0a, 0x04, 0x54, 0x65, 0x73, 0x74, - 0x12, 0x10, 0x0a, 0x03, 0x6e, 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x03, 0x6e, - 0x75, 0x6d, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x73, 0x66, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x03, 0x6d, 0x73, 0x66, 0x42, 0x06, 0x5a, 0x04, 0x2e, 0x2f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x33, + 0x72, 0x6f, 0x74, 0x6f, 0x74, 0x65, 0x73, 0x74, 0x1a, 0x08, 0x68, 0x69, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x22, 0x80, 0x01, 0x0a, 0x04, 0x54, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6e, + 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x03, 0x6e, 0x75, 0x6d, 0x12, 0x10, 0x0a, + 0x03, 0x6d, 0x73, 0x66, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6d, 0x73, 0x66, 0x12, + 0x20, 0x0a, 0x03, 0x66, 0x6f, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x74, 0x65, 0x73, 0x74, 0x2e, 0x46, 0x6f, 0x6f, 0x52, 0x03, 0x66, 0x6f, + 0x6f, 0x12, 0x32, 0x0a, 0x09, 0x68, 0x69, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x74, 0x65, 0x73, 0x74, + 0x2e, 0x48, 0x69, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x52, 0x09, 0x68, 0x69, 0x43, 0x6f, + 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x22, 0x45, 0x0a, 0x03, 0x46, 0x6f, 0x6f, 0x12, 0x12, 0x0a, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x42, 0x03, 0x5a, 0x01, + 0x2e, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -116,16 +205,20 @@ func file_hello_proto_rawDescGZIP() []byte { return file_hello_proto_rawDescData } -var file_hello_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_hello_proto_msgTypes = make([]protoimpl.MessageInfo, 2) var file_hello_proto_goTypes = []interface{}{ - (*Test)(nil), // 0: prototest.Test + (*Test)(nil), // 0: prototest.Test + (*Foo)(nil), // 1: prototest.Foo + (*HiContent)(nil), // 2: prototest.HiContent } var file_hello_proto_depIdxs = []int32{ - 0, // [0:0] is the sub-list for method output_type - 0, // [0:0] is the sub-list for method input_type - 0, // [0:0] is the sub-list for extension type_name - 0, // [0:0] is the sub-list for extension extendee - 0, // [0:0] is the sub-list for field type_name + 1, // 0: prototest.Test.foo:type_name -> prototest.Foo + 2, // 1: prototest.Test.hiContent:type_name -> prototest.HiContent + 2, // [2:2] is the sub-list for method output_type + 2, // [2:2] is the sub-list for method input_type + 2, // [2:2] is the sub-list for extension type_name + 2, // [2:2] is the sub-list for extension extendee + 0, // [0:2] is the sub-list for field type_name } func init() { file_hello_proto_init() } @@ -133,6 +226,7 @@ func file_hello_proto_init() { if File_hello_proto != nil { return } + file_hi_proto_init() if !protoimpl.UnsafeEnabled { file_hello_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Test); i { @@ -146,6 +240,18 @@ func file_hello_proto_init() { return nil } } + file_hello_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Foo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ @@ -153,7 +259,7 @@ func file_hello_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_hello_proto_rawDesc, NumEnums: 0, - NumMessages: 1, + NumMessages: 2, NumExtensions: 0, NumServices: 0, }, diff --git a/integration-tests/pb/hello.proto b/integration-tests/pb/hello.proto index 3a67c3b4b5..ca6a4ff9e8 100644 --- a/integration-tests/pb/hello.proto +++ b/integration-tests/pb/hello.proto @@ -19,9 +19,19 @@ syntax = "proto3"; package prototest; -option go_package = "./pb"; +option go_package = "."; + +import "hi.proto"; message Test { int32 num = 1; string msf = 2; + Foo foo = 3; + HiContent hiContent = 4; +} + +message Foo { + string name = 1; + int32 value = 2; + repeated string names = 3; } diff --git a/integration-tests/pb/hi.pb.go b/integration-tests/pb/hi.pb.go new file mode 100644 index 0000000000..1027caf894 --- /dev/null +++ b/integration-tests/pb/hi.pb.go @@ -0,0 +1,168 @@ +//* +// 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. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.30.0 +// protoc v3.21.12 +// source: hi.proto + +package __ + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type HiContent struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id int32 `protobuf:"varint,2,opt,name=id,proto3" json:"id,omitempty"` + Content string `protobuf:"bytes,1,opt,name=content,proto3" json:"content,omitempty"` +} + +func (x *HiContent) Reset() { + *x = HiContent{} + if protoimpl.UnsafeEnabled { + mi := &file_hi_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *HiContent) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HiContent) ProtoMessage() {} + +func (x *HiContent) ProtoReflect() protoreflect.Message { + mi := &file_hi_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use HiContent.ProtoReflect.Descriptor instead. +func (*HiContent) Descriptor() ([]byte, []int) { + return file_hi_proto_rawDescGZIP(), []int{0} +} + +func (x *HiContent) GetId() int32 { + if x != nil { + return x.Id + } + return 0 +} + +func (x *HiContent) GetContent() string { + if x != nil { + return x.Content + } + return "" +} + +var File_hi_proto protoreflect.FileDescriptor + +var file_hi_proto_rawDesc = []byte{ + 0x0a, 0x08, 0x68, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x09, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x74, 0x65, 0x73, 0x74, 0x22, 0x35, 0x0a, 0x09, 0x48, 0x69, 0x43, 0x6f, 0x6e, 0x74, 0x65, + 0x6e, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x02, + 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x42, 0x03, 0x5a, 0x01, + 0x2e, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_hi_proto_rawDescOnce sync.Once + file_hi_proto_rawDescData = file_hi_proto_rawDesc +) + +func file_hi_proto_rawDescGZIP() []byte { + file_hi_proto_rawDescOnce.Do(func() { + file_hi_proto_rawDescData = protoimpl.X.CompressGZIP(file_hi_proto_rawDescData) + }) + return file_hi_proto_rawDescData +} + +var file_hi_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_hi_proto_goTypes = []interface{}{ + (*HiContent)(nil), // 0: prototest.HiContent +} +var file_hi_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_hi_proto_init() } +func file_hi_proto_init() { + if File_hi_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_hi_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*HiContent); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_hi_proto_rawDesc, + NumEnums: 0, + NumMessages: 1, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_hi_proto_goTypes, + DependencyIndexes: file_hi_proto_depIdxs, + MessageInfos: file_hi_proto_msgTypes, + }.Build() + File_hi_proto = out.File + file_hi_proto_rawDesc = nil + file_hi_proto_goTypes = nil + file_hi_proto_depIdxs = nil +} diff --git a/integration-tests/pb/hi.proto b/integration-tests/pb/hi.proto new file mode 100644 index 0000000000..f5a401a54d --- /dev/null +++ b/integration-tests/pb/hi.proto @@ -0,0 +1,9 @@ +syntax = "proto3"; +package prototest; + +option go_package = "."; + +message HiContent { + int32 id = 2; + string content = 1; +} diff --git a/pulsar/schema.go b/pulsar/schema.go index 997d85fb89..405049dcee 100644 --- a/pulsar/schema.go +++ b/pulsar/schema.go @@ -30,6 +30,9 @@ import ( "github.com/linkedin/goavro/v2" "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/reflect/protodesc" + "google.golang.org/protobuf/reflect/protoreflect" + "google.golang.org/protobuf/types/descriptorpb" ) type SchemaType int @@ -55,6 +58,7 @@ const ( AUTO = -2 // AutoConsume = -3 //Auto Consume Type. AutoPublish = -4 // Auto Publish Type. + ProtoNative = 20 //Protobuf native message encoding and decoding ) // Encapsulates data around the schema definition @@ -102,6 +106,8 @@ func NewSchema(schemaType SchemaType, schemaData []byte, properties map[string]s s = NewFloatSchema(properties) case DOUBLE: s = NewDoubleSchema(properties) + case ProtoNative: + s = newProtoNativeSchema(schemaDef, properties) default: err = fmt.Errorf("not support schema type of %v", schemaType) } @@ -219,6 +225,84 @@ func (ps *ProtoSchema) GetSchemaInfo() *SchemaInfo { return &ps.SchemaInfo } +type ProtoNativeSchema struct { + SchemaInfo +} + +func NewProtoNativeSchemaWithMessage(message proto.Message, properties map[string]string) *ProtoNativeSchema { + schemaDef, err := getProtoNativeSchemaInfo(message) + if err != nil { + log.Fatalf("Get ProtoNative schema info error:%v", err) + } + return newProtoNativeSchema(schemaDef, properties) +} + +func newProtoNativeSchema(protoNativeSchemaDef string, properties map[string]string) *ProtoNativeSchema { + pns := new(ProtoNativeSchema) + pns.SchemaInfo.Schema = protoNativeSchemaDef + pns.SchemaInfo.Type = ProtoNative + pns.SchemaInfo.Properties = properties + pns.SchemaInfo.Name = "ProtoNative" + return pns +} + +func getProtoNativeSchemaInfo(message proto.Message) (string, error) { + fileDesc := message.ProtoReflect().Descriptor().ParentFile() + fileProtoMap := make(map[string]*descriptorpb.FileDescriptorProto) + getFileProto(fileDesc, fileProtoMap) + + fileDescList := make([]*descriptorpb.FileDescriptorProto, 0, len(fileProtoMap)) + for _, v := range fileProtoMap { + fileDescList = append(fileDescList, v) + } + fileDescSet := descriptorpb.FileDescriptorSet{ + File: fileDescList, + } + bytesData, err := proto.Marshal(&fileDescSet) + if err != nil { + return "", err + } + schemaData := ProtoNativeSchemaData{ + FileDescriptorSet: bytesData, + RootMessageTypeName: string(message.ProtoReflect().Descriptor().FullName()), + RootFileDescriptorName: fileDesc.Path(), + } + jsonData, err := json.Marshal(schemaData) + if err != nil { + return "", err + } + return string(jsonData), nil +} + +type ProtoNativeSchemaData struct { + FileDescriptorSet []byte `json:"fileDescriptorSet"` + RootMessageTypeName string `json:"rootMessageTypeName"` + RootFileDescriptorName string `json:"rootFileDescriptorName"` +} + +func getFileProto(fileDesc protoreflect.FileDescriptor, protoMap map[string]*descriptorpb.FileDescriptorProto) { + for i := 0; i < fileDesc.Imports().Len(); i++ { + getFileProto(fileDesc.Imports().Get(i).ParentFile(), protoMap) + } + protoMap[fileDesc.Path()] = protodesc.ToFileDescriptorProto(fileDesc) +} + +func (ps *ProtoNativeSchema) Encode(data interface{}) ([]byte, error) { + return proto.Marshal(data.(proto.Message)) +} + +func (ps *ProtoNativeSchema) Decode(data []byte, v interface{}) error { + return proto.Unmarshal(data, v.(proto.Message)) +} + +func (ps *ProtoNativeSchema) Validate(message []byte) error { + return ps.Decode(message, nil) +} + +func (ps *ProtoNativeSchema) GetSchemaInfo() *SchemaInfo { + return &ps.SchemaInfo +} + type AvroSchema struct { AvroCodec SchemaInfo diff --git a/pulsar/schema_test.go b/pulsar/schema_test.go index e7d52ca0d3..c2008f6de9 100644 --- a/pulsar/schema_test.go +++ b/pulsar/schema_test.go @@ -22,7 +22,7 @@ import ( "log" "testing" - "github.com/apache/pulsar-client-go/integration-tests/pb" + pb "github.com/apache/pulsar-client-go/integration-tests/pb" "github.com/stretchr/testify/assert" ) @@ -147,6 +147,69 @@ func TestProtoSchema(t *testing.T) { defer consumer.Close() } +func TestProtoNativeSchema(t *testing.T) { + client := createClient() + defer client.Close() + + topic := "proto-native" + + // create producer + psProducer := NewProtoNativeSchemaWithMessage(&pb.Test{}, nil) + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + Schema: psProducer, + }) + assert.Nil(t, err) + + names := []string{ + "name-a", + "name-b", + "name-c", + } + if _, err := producer.Send(context.Background(), &ProducerMessage{ + Value: &pb.Test{ + Num: 100, + Msf: "pulsar", + Foo: &pb.Foo{ + Name: "foo name", + Value: 200, + Names: names, + }, + HiContent: &pb.HiContent{ + Id: 300, + Content: "hi content", + }, + }, + }); err != nil { + log.Fatal(err) + } + + //create consumer + unobj := pb.Test{} + psConsumer := NewProtoNativeSchemaWithMessage(&pb.Test{}, nil) + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "sub-1", + Schema: psConsumer, + SubscriptionInitialPosition: SubscriptionPositionEarliest, + }) + assert.Nil(t, err) + + msg, err := consumer.Receive(context.Background()) + assert.Nil(t, err) + err = msg.GetSchemaValue(&unobj) + assert.Nil(t, err) + assert.Equal(t, unobj.Num, int32(100)) + assert.Equal(t, unobj.Msf, "pulsar") + assert.Equal(t, unobj.Foo.Name, "foo name") + assert.Equal(t, unobj.Foo.Value, int32(200)) + assert.Equal(t, unobj.Foo.Names, names) + assert.Equal(t, unobj.HiContent.Id, int32(300)) + assert.Equal(t, unobj.HiContent.Content, "hi content") + consumer.Ack(msg) + defer consumer.Close() +} + func TestAvroSchema(t *testing.T) { client := createClient() defer client.Close() From 2437caa14fb0e2f4ab34eaee08e9a83612244d0f Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Sat, 6 May 2023 09:53:12 +0800 Subject: [PATCH 047/348] [feat][txn]Implement transactional consumer/producer API (#1002) * [feat][txn]Implement transactional consumer/producer API * rollback * fix test * optimize code * state check * fix callback * golangci-lint * add more test --- pulsar/client.go | 11 + pulsar/client_impl.go | 8 + pulsar/consumer.go | 3 + pulsar/consumer_impl.go | 10 + pulsar/consumer_multitopic.go | 17 ++ pulsar/consumer_partition.go | 174 ++++++++++--- pulsar/consumer_regex.go | 18 ++ pulsar/helper_for_test.go | 2 +- pulsar/internal/batch_builder.go | 8 + pulsar/internal/commands.go | 12 +- pulsar/internal/http_client.go | 4 +- pulsar/internal/key_based_batch_builder.go | 5 +- .../consumer_interceptor_test.go | 4 + pulsar/message.go | 4 + pulsar/producer_partition.go | 110 ++++++-- pulsar/transaction.go | 8 +- pulsar/transaction_coordinator_client.go | 18 +- pulsar/transaction_impl.go | 2 + pulsar/transaction_test.go | 240 +++++++++++++++++- 19 files changed, 573 insertions(+), 85 deletions(-) diff --git a/pulsar/client.go b/pulsar/client.go index bc3f4f5892..7e6725d42e 100644 --- a/pulsar/client.go +++ b/pulsar/client.go @@ -184,6 +184,17 @@ type Client interface { // {@link Consumer} or {@link Producer} instances directly on a particular partition. TopicPartitions(topic string) ([]string, error) + // NewTransaction creates a new Transaction instance. + // + // This function is used to initiate a new transaction for performing + // atomic operations on the message broker. It returns a Transaction + // object that can be used to produce, consume and commit messages in a + // transactional manner. + // + // In case of any errors while creating the transaction, an error will + // be returned. + NewTransaction(duration time.Duration) (Transaction, error) + // Close Closes the Client and free associated resources Close() } diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index 5322597569..9b44987949 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -195,6 +195,14 @@ func newClient(options ClientOptions) (Client, error) { return c, nil } +func (c *client) NewTransaction(timeout time.Duration) (Transaction, error) { + id, err := c.tcClient.newTransaction(timeout) + if err != nil { + return nil, err + } + return newTransaction(*id, c.tcClient, timeout), nil +} + func (c *client) CreateProducer(options ProducerOptions) (Producer, error) { producer, err := newProducer(c, &options) if err == nil { diff --git a/pulsar/consumer.go b/pulsar/consumer.go index 64a096d50c..3ef72c7ca8 100644 --- a/pulsar/consumer.go +++ b/pulsar/consumer.go @@ -265,6 +265,9 @@ type Consumer interface { // AckID the consumption of a single message, identified by its MessageID AckID(MessageID) error + // AckWithTxn the consumption of a single message with a transaction + AckWithTxn(Message, Transaction) error + // AckCumulative the reception of all the messages in the stream up to (and including) // the provided message. AckCumulative(msg Message) error diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index fd7fa5781d..07f38c3633 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -38,6 +38,7 @@ type acker interface { // AckID does not handle errors returned by the Broker side, so no need to wait for doneCh to finish. AckID(id MessageID) error AckIDWithResponse(id MessageID) error + AckIDWithTxn(msgID MessageID, txn Transaction) error AckIDCumulative(msgID MessageID) error AckIDWithResponseCumulative(msgID MessageID) error NackID(id MessageID) @@ -478,6 +479,15 @@ func (c *consumer) Receive(ctx context.Context) (message Message, err error) { } } +func (c *consumer) AckWithTxn(msg Message, txn Transaction) error { + msgID := msg.ID() + if err := c.checkMsgIDPartition(msgID); err != nil { + return err + } + + return c.consumers[msgID.PartitionIdx()].AckIDWithTxn(msgID, txn) +} + // Chan return the message chan to users func (c *consumer) Chan() <-chan ConsumerMessage { return c.messageCh diff --git a/pulsar/consumer_multitopic.go b/pulsar/consumer_multitopic.go index 8108c294b5..f6630dd65f 100644 --- a/pulsar/consumer_multitopic.go +++ b/pulsar/consumer_multitopic.go @@ -143,6 +143,23 @@ func (c *multiTopicConsumer) AckID(msgID MessageID) error { return mid.consumer.AckID(msgID) } +// AckWithTxn the consumption of a single message with a transaction +func (c *multiTopicConsumer) AckWithTxn(msg Message, txn Transaction) error { + msgID := msg.ID() + if !checkMessageIDType(msgID) { + c.log.Warnf("invalid message id type %T", msgID) + return errors.New("invalid message id type in multi_consumer") + } + mid := toTrackingMessageID(msgID) + + if mid.consumer == nil { + c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID) + return errors.New("unable to ack message because consumer is nil") + } + + return mid.consumer.AckIDWithTxn(msgID, txn) +} + // AckCumulative the reception of all the messages in the stream up to (and including) // the provided message func (c *multiTopicConsumer) AckCumulative(msg Message) error { diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index a8ba7b5540..6e241d8f7f 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -417,6 +417,118 @@ func (pc *partitionConsumer) Unsubscribe() error { return req.err } +// ackIDCommon handles common logic for acknowledging messages with or without transactions. +// withTxn should be set to true when dealing with transactions. +func (pc *partitionConsumer) ackIDCommon(msgID MessageID, withResponse bool, txn Transaction) error { + if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing { + pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer") + return errors.New("consumer state is closed") + } + + if cmid, ok := msgID.(*chunkMessageID); ok { + return pc.unAckChunksTracker.ack(cmid) + } + + trackingID := toTrackingMessageID(msgID) + + if trackingID != nil && trackingID.ack() { + // All messages in the same batch have been acknowledged, we only need to acknowledge the + // MessageID that represents the entry that stores the whole batch + trackingID = &trackingMessageID{ + messageID: &messageID{ + ledgerID: trackingID.ledgerID, + entryID: trackingID.entryID, + }, + } + pc.metrics.AcksCounter.Inc() + pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9) + } else if !pc.options.enableBatchIndexAck { + return nil + } + + var err error + if withResponse { + if txn != nil { + ackReq := pc.sendIndividualAckWithTxn(trackingID, txn.(*transaction)) + <-ackReq.doneCh + err = ackReq.err + } else { + ackReq := pc.sendIndividualAck(trackingID) + <-ackReq.doneCh + err = ackReq.err + } + } else { + pc.ackGroupingTracker.add(trackingID) + } + pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID) + return err +} + +// AckIDWithTxn acknowledges the consumption of a message with transaction. +func (pc *partitionConsumer) AckIDWithTxn(msgID MessageID, txn Transaction) error { + return pc.ackIDCommon(msgID, true, txn) +} + +// ackID acknowledges the consumption of a message and optionally waits for response from the broker. +func (pc *partitionConsumer) ackID(msgID MessageID, withResponse bool) error { + return pc.ackIDCommon(msgID, withResponse, nil) +} + +func (pc *partitionConsumer) internalAckWithTxn(req *ackWithTxnRequest) { + defer close(req.doneCh) + if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing { + pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer") + req.err = newError(ConsumerClosed, "Failed to ack by closing or closed consumer") + return + } + if req.Transaction.state != TxnOpen { + pc.log.WithField("state", req.Transaction.state).Error("Failed to ack by a non-open transaction.") + req.err = newError(InvalidStatus, "Failed to ack by a non-open transaction.") + return + } + msgID := req.msgID + + messageIDs := make([]*pb.MessageIdData, 1) + messageIDs[0] = &pb.MessageIdData{ + LedgerId: proto.Uint64(uint64(msgID.ledgerID)), + EntryId: proto.Uint64(uint64(msgID.entryID)), + } + if pc.options.enableBatchIndexAck && msgID.tracker != nil { + ackSet := msgID.tracker.toAckSet() + if ackSet != nil { + messageIDs[0].AckSet = ackSet + } + } + + reqID := pc.client.rpcClient.NewRequestID() + txnID := req.Transaction.GetTxnID() + cmdAck := &pb.CommandAck{ + ConsumerId: proto.Uint64(pc.consumerID), + MessageId: messageIDs, + AckType: pb.CommandAck_Individual.Enum(), + TxnidMostBits: proto.Uint64(txnID.MostSigBits), + TxnidLeastBits: proto.Uint64(txnID.LeastSigBits), + } + + if err := req.Transaction.registerAckTopic(pc.options.topic, pc.options.subscription); err != nil { + req.err = err + return + } + + if err := req.Transaction.registerSendOrAckOp(); err != nil { + req.err = err + return + } + + cmdAck.RequestId = proto.Uint64(reqID) + _, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), reqID, pb.BaseCommand_ACK, cmdAck) + if err != nil { + pc.log.WithError(err).Error("Ack with response error") + } + req.Transaction.endSendOrAckOp(err) + req.err = err +} + func (pc *partitionConsumer) internalUnsubscribe(unsub *unsubscribeRequest) { defer close(unsub.doneCh) @@ -488,47 +600,6 @@ func (pc *partitionConsumer) requestGetLastMessageID() (*trackingMessageID, erro return convertToMessageID(id), nil } -func (pc *partitionConsumer) ackID(msgID MessageID, withResponse bool) error { - if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing { - pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer") - return errors.New("consumer state is closed") - } - - if cmid, ok := msgID.(*chunkMessageID); ok { - return pc.unAckChunksTracker.ack(cmid) - } - - trackingID := toTrackingMessageID(msgID) - - if trackingID != nil && trackingID.ack() { - // All messages in the same batch have been acknowledged, we only need to acknowledge the - // MessageID that represents the entry that stores the whole batch - trackingID = &trackingMessageID{ - messageID: &messageID{ - ledgerID: trackingID.ledgerID, - entryID: trackingID.entryID, - }, - } - pc.metrics.AcksCounter.Inc() - pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9) - } else if !pc.options.enableBatchIndexAck { - return nil - } - - var ackReq *ackRequest - if withResponse { - ackReq := pc.sendIndividualAck(trackingID) - <-ackReq.doneCh - } else { - pc.ackGroupingTracker.add(trackingID) - } - pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID) - if ackReq == nil { - return nil - } - return ackReq.err -} - func (pc *partitionConsumer) sendIndividualAck(msgID MessageID) *ackRequest { ackReq := &ackRequest{ doneCh: make(chan struct{}), @@ -539,6 +610,17 @@ func (pc *partitionConsumer) sendIndividualAck(msgID MessageID) *ackRequest { return ackReq } +func (pc *partitionConsumer) sendIndividualAckWithTxn(msgID MessageID, txn *transaction) *ackWithTxnRequest { + ackReq := &ackWithTxnRequest{ + Transaction: txn, + doneCh: make(chan struct{}), + ackType: individualAck, + msgID: *msgID.(*trackingMessageID), + } + pc.eventsCh <- ackReq + return ackReq +} + func (pc *partitionConsumer) AckIDWithResponse(msgID MessageID) error { if !checkMessageIDType(msgID) { pc.log.Errorf("invalid message id type %T", msgID) @@ -1389,6 +1471,14 @@ type ackRequest struct { err error } +type ackWithTxnRequest struct { + doneCh chan struct{} + msgID trackingMessageID + Transaction *transaction + ackType int + err error +} + type unsubscribeRequest struct { doneCh chan struct{} err error @@ -1444,6 +1534,8 @@ func (pc *partitionConsumer) runEventsLoop() { switch v := i.(type) { case *ackRequest: pc.internalAck(v) + case *ackWithTxnRequest: + pc.internalAckWithTxn(v) case []*pb.MessageIdData: pc.internalAckList(v) case *redeliveryRequest: diff --git a/pulsar/consumer_regex.go b/pulsar/consumer_regex.go index 2520af5c8e..79e2293b49 100644 --- a/pulsar/consumer_regex.go +++ b/pulsar/consumer_regex.go @@ -193,6 +193,24 @@ func (c *regexConsumer) AckID(msgID MessageID) error { return mid.consumer.AckID(msgID) } +// AckID the consumption of a single message, identified by its MessageID +func (c *regexConsumer) AckWithTxn(msg Message, txn Transaction) error { + msgID := msg.ID() + if !checkMessageIDType(msgID) { + c.log.Warnf("invalid message id type %T", msgID) + return fmt.Errorf("invalid message id type %T", msgID) + } + + mid := toTrackingMessageID(msgID) + + if mid.consumer == nil { + c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID) + return errors.New("consumer is nil in consumer_regex") + } + + return mid.consumer.AckIDWithTxn(msgID, txn) +} + // AckCumulative the reception of all the messages in the stream up to (and including) // the provided message. func (c *regexConsumer) AckCumulative(msg Message) error { diff --git a/pulsar/helper_for_test.go b/pulsar/helper_for_test.go index 7bbf66e9fd..426855b26f 100644 --- a/pulsar/helper_for_test.go +++ b/pulsar/helper_for_test.go @@ -159,7 +159,7 @@ func topicStats(topic string) (map[string]interface{}, error) { func transactionStats(id *TxnID) (map[string]interface{}, error) { var metadata map[string]interface{} - path := fmt.Sprintf("admin/v3/transactions/transactionMetadata/%d/%d", id.mostSigBits, id.leastSigBits) + path := fmt.Sprintf("admin/v3/transactions/transactionMetadata/%d/%d", id.MostSigBits, id.LeastSigBits) err := httpGet(path, &metadata) return metadata, err } diff --git a/pulsar/internal/batch_builder.go b/pulsar/internal/batch_builder.go index 649aba4844..6df3a61850 100644 --- a/pulsar/internal/batch_builder.go +++ b/pulsar/internal/batch_builder.go @@ -51,6 +51,9 @@ type BatchBuilder interface { payload []byte, callback interface{}, replicateTo []string, deliverAt time.Time, schemaVersion []byte, multiSchemaEnabled bool, + useTxn bool, + mostSigBits uint64, + leastSigBits uint64, ) bool // Flush all the messages buffered in the client and wait until all messages have been successfully persisted. @@ -185,6 +188,7 @@ func (bc *batchContainer) Add( payload []byte, callback interface{}, replicateTo []string, deliverAt time.Time, schemaVersion []byte, multiSchemaEnabled bool, + useTxn bool, mostSigBits uint64, leastSigBits uint64, ) bool { if replicateTo != nil && bc.numMessages != 0 { @@ -223,6 +227,10 @@ func (bc *batchContainer) Add( } bc.cmdSend.Send.SequenceId = proto.Uint64(sequenceID) + if useTxn { + bc.cmdSend.Send.TxnidMostBits = proto.Uint64(mostSigBits) + bc.cmdSend.Send.TxnidLeastBits = proto.Uint64(leastSigBits) + } } addSingleMessageToBatch(bc.buffer, metadata, payload) diff --git a/pulsar/internal/commands.go b/pulsar/internal/commands.go index 00e075beee..7471ee0d46 100644 --- a/pulsar/internal/commands.go +++ b/pulsar/internal/commands.go @@ -22,11 +22,10 @@ import ( "errors" "fmt" - "google.golang.org/protobuf/proto" - "github.com/apache/pulsar-client-go/pulsar/internal/compression" "github.com/apache/pulsar-client-go/pulsar/internal/crypto" pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" + "google.golang.org/protobuf/proto" ) const ( @@ -332,7 +331,10 @@ func SingleSend(wb Buffer, msgMetadata *pb.MessageMetadata, compressedPayload Buffer, encryptor crypto.Encryptor, - maxMassageSize uint32) error { + maxMassageSize uint32, + useTxn bool, + mostSigBits uint64, + leastSigBits uint64) error { cmdSend := baseCommand( pb.BaseCommand_SEND, &pb.CommandSend{ @@ -344,6 +346,10 @@ func SingleSend(wb Buffer, isChunk := true cmdSend.Send.IsChunk = &isChunk } + if useTxn { + cmdSend.Send.TxnidMostBits = proto.Uint64(mostSigBits) + cmdSend.Send.TxnidLeastBits = proto.Uint64(leastSigBits) + } // payload has been compressed so compressionProvider can be nil return serializeMessage(wb, cmdSend, msgMetadata, compressedPayload, nil, encryptor, maxMassageSize, false) diff --git a/pulsar/internal/http_client.go b/pulsar/internal/http_client.go index c3b38f26ff..2d440d6f9a 100644 --- a/pulsar/internal/http_client.go +++ b/pulsar/internal/http_client.go @@ -190,7 +190,9 @@ func (c *httpClient) GetWithOptions(endpoint string, obj interface{}, params map req.params = query } - resp, err := checkSuccessful(c.doRequest(req)) + doRequest, err := c.doRequest(req) + defer safeRespClose(doRequest) + resp, err := checkSuccessful(doRequest, err) if err != nil { return nil, err } diff --git a/pulsar/internal/key_based_batch_builder.go b/pulsar/internal/key_based_batch_builder.go index 334e6746ab..88a4d5ed01 100644 --- a/pulsar/internal/key_based_batch_builder.go +++ b/pulsar/internal/key_based_batch_builder.go @@ -132,6 +132,9 @@ func (bc *keyBasedBatchContainer) Add( payload []byte, callback interface{}, replicateTo []string, deliverAt time.Time, schemaVersion []byte, multiSchemaEnabled bool, + useTxn bool, + mostSigBits uint64, + leastSigBits uint64, ) bool { if replicateTo != nil && bc.numMessages != 0 { // If the current batch is not empty and we're trying to set the replication clusters, @@ -162,7 +165,7 @@ func (bc *keyBasedBatchContainer) Add( add := batchPart.Add( metadata, sequenceIDGenerator, payload, callback, replicateTo, deliverAt, - schemaVersion, multiSchemaEnabled, + schemaVersion, multiSchemaEnabled, useTxn, mostSigBits, leastSigBits, ) if !add { return false diff --git a/pulsar/internal/pulsartracing/consumer_interceptor_test.go b/pulsar/internal/pulsartracing/consumer_interceptor_test.go index 34e09d58a9..06c9a588ff 100644 --- a/pulsar/internal/pulsartracing/consumer_interceptor_test.go +++ b/pulsar/internal/pulsartracing/consumer_interceptor_test.go @@ -52,6 +52,10 @@ func (c *mockConsumer) Subscription() string { return "" } +func (c *mockConsumer) AckWithTxn(msg pulsar.Message, txn pulsar.Transaction) error { + return nil +} + func (c *mockConsumer) Unsubscribe() error { return nil } diff --git a/pulsar/message.go b/pulsar/message.go index 98190e918b..83afd3f779 100644 --- a/pulsar/message.go +++ b/pulsar/message.go @@ -69,6 +69,10 @@ type ProducerMessage struct { //Schema assign to the current message //Note: messages may have a different schema from producer schema, use it instead of producer schema when assigned Schema Schema + + //Transaction assign to the current message + //Note: The message is not visible before the transaction is committed. + Transaction Transaction } // Message abstraction used in Pulsar diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 744df7995b..e77c929c3c 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -663,6 +663,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { chunkID: chunkID, uuid: uuid, chunkRecorder: cr, + transaction: request.transaction, } // the permit of first chunk has acquired if chunkID != 0 && !p.canAddToQueue(nsr, 0) { @@ -681,16 +682,16 @@ func (p *partitionProducer) internalSend(request *sendRequest) { } else { smm := p.genSingleMessageMetadataInBatch(msg, uncompressedSize) multiSchemaEnabled := !p.options.DisableMultiSchema - added := p.batchBuilder.Add(smm, p.sequenceIDGenerator, uncompressedPayload, request, - msg.ReplicationClusters, deliverAt, schemaVersion, multiSchemaEnabled) + added := addRequestToBatch(smm, p, uncompressedPayload, request, msg, deliverAt, schemaVersion, + multiSchemaEnabled) if !added { - // The current batch is full.. flush it and retry + // The current batch is full. flush it and retry p.internalFlushCurrentBatch() // after flushing try again to add the current payload - if ok := p.batchBuilder.Add(smm, p.sequenceIDGenerator, uncompressedPayload, request, - msg.ReplicationClusters, deliverAt, schemaVersion, multiSchemaEnabled); !ok { + if ok := addRequestToBatch(smm, p, uncompressedPayload, request, msg, deliverAt, schemaVersion, + multiSchemaEnabled); !ok { p.releaseSemaphoreAndMem(uncompressedPayloadSize) request.callback(nil, request.msg, errFailAddToBatch) p.log.WithField("size", uncompressedSize). @@ -707,6 +708,23 @@ func (p *partitionProducer) internalSend(request *sendRequest) { } } +func addRequestToBatch(smm *pb.SingleMessageMetadata, p *partitionProducer, + uncompressedPayload []byte, + request *sendRequest, msg *ProducerMessage, deliverAt time.Time, + schemaVersion []byte, multiSchemaEnabled bool) bool { + var ok bool + if request.transaction != nil { + txnID := request.transaction.GetTxnID() + ok = p.batchBuilder.Add(smm, p.sequenceIDGenerator, uncompressedPayload, request, + msg.ReplicationClusters, deliverAt, schemaVersion, multiSchemaEnabled, true, txnID.MostSigBits, + txnID.LeastSigBits) + } else { + ok = p.batchBuilder.Add(smm, p.sequenceIDGenerator, uncompressedPayload, request, + msg.ReplicationClusters, deliverAt, schemaVersion, multiSchemaEnabled, false, 0, 0) + } + return ok +} + func (p *partitionProducer) genMetadata(msg *ProducerMessage, uncompressedSize int, deliverAt time.Time) (mm *pb.MessageMetadata) { @@ -789,16 +807,36 @@ func (p *partitionProducer) internalSingleSend(mm *pb.MessageMetadata, } sid := *mm.SequenceId - - if err := internal.SingleSend( - buffer, - p.producerID, - sid, - mm, - payloadBuf, - p.encryptor, - maxMessageSize, - ); err != nil { + var err error + if request.transaction != nil { + txnID := request.transaction.GetTxnID() + err = internal.SingleSend( + buffer, + p.producerID, + sid, + mm, + payloadBuf, + p.encryptor, + maxMessageSize, + true, + txnID.MostSigBits, + txnID.LeastSigBits, + ) + } else { + err = internal.SingleSend( + buffer, + p.producerID, + sid, + mm, + payloadBuf, + p.encryptor, + maxMessageSize, + false, + 0, + 0, + ) + } + if err != nil { request.callback(nil, request.msg, err) p.releaseSemaphoreAndMem(int64(len(msg.Payload))) p.log.WithError(err).Errorf("Single message serialize failed %s", msg.Value) @@ -952,6 +990,9 @@ func (p *partitionProducer) failTimeoutMessages() { sr.callback(nil, sr.msg, errSendTimeout) }) } + if sr.transaction != nil { + sr.transaction.endSendOrAckOp(nil) + } } // flag the send has completed with error, flush make no effect @@ -1067,9 +1108,34 @@ func (p *partitionProducer) SendAsync(ctx context.Context, msg *ProducerMessage, func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *ProducerMessage, callback func(MessageID, *ProducerMessage, error), flushImmediately bool) { + //Register transaction operation to transaction and the transaction coordinator. + var newCallback func(MessageID, *ProducerMessage, error) + if msg.Transaction != nil { + transactionImpl := (msg.Transaction).(*transaction) + if transactionImpl.state != TxnOpen { + p.log.WithField("state", transactionImpl.state).Error("Failed to send message" + + " by a non-open transaction.") + callback(nil, msg, newError(InvalidStatus, "Failed to send message by a non-open transaction.")) + return + } + + if err := transactionImpl.registerProducerTopic(p.topic); err != nil { + callback(nil, msg, err) + return + } + if err := transactionImpl.registerSendOrAckOp(); err != nil { + callback(nil, msg, err) + } + newCallback = func(id MessageID, producerMessage *ProducerMessage, err error) { + callback(id, producerMessage, err) + transactionImpl.endSendOrAckOp(err) + } + } else { + newCallback = callback + } if p.getProducerState() != producerReady { // Producer is closing - callback(nil, msg, errProducerClosed) + newCallback(nil, msg, errProducerClosed) return } @@ -1078,16 +1144,20 @@ func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *Producer // callbackOnce make sure the callback is only invoked once in chunking callbackOnce := &sync.Once{} - + var txn *transaction + if msg.Transaction != nil { + txn = (msg.Transaction).(*transaction) + } sr := &sendRequest{ ctx: ctx, msg: msg, - callback: callback, + callback: newCallback, callbackOnce: callbackOnce, flushImmediately: flushImmediately, publishTime: time.Now(), blockCh: bc, closeBlockChOnce: &sync.Once{}, + transaction: txn, } p.options.Interceptors.BeforeSend(p, msg) @@ -1191,6 +1261,9 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) p.options.Interceptors.OnSendAcknowledgement(p, sr.msg, msgID) } } + if sr.transaction != nil { + sr.transaction.endSendOrAckOp(nil) + } } // Mark this pending item as done @@ -1287,6 +1360,7 @@ type sendRequest struct { chunkID int uuid string chunkRecorder *chunkRecorder + transaction *transaction } // stopBlock can be invoked multiple times safety diff --git a/pulsar/transaction.go b/pulsar/transaction.go index 60e1d2bf2b..944c7e339d 100644 --- a/pulsar/transaction.go +++ b/pulsar/transaction.go @@ -49,10 +49,10 @@ const ( // TxnID An identifier for representing a transaction. type TxnID struct { - // mostSigBits The most significant 64 bits of this TxnID. - mostSigBits uint64 - // leastSigBits The least significant 64 bits of this TxnID. - leastSigBits uint64 + // MostSigBits The most significant 64 bits of this TxnID. + MostSigBits uint64 + // LeastSigBits The least significant 64 bits of this TxnID. + LeastSigBits uint64 } // Transaction used to guarantee exactly-once diff --git a/pulsar/transaction_coordinator_client.go b/pulsar/transaction_coordinator_client.go index 1535fad113..96cca8708b 100644 --- a/pulsar/transaction_coordinator_client.go +++ b/pulsar/transaction_coordinator_client.go @@ -135,11 +135,11 @@ func (tc *transactionCoordinatorClient) addPublishPartitionToTxn(id *TxnID, part requestID := tc.client.rpcClient.NewRequestID() cmdAddPartitions := &pb.CommandAddPartitionToTxn{ RequestId: proto.Uint64(requestID), - TxnidMostBits: proto.Uint64(id.mostSigBits), - TxnidLeastBits: proto.Uint64(id.leastSigBits), + TxnidMostBits: proto.Uint64(id.MostSigBits), + TxnidLeastBits: proto.Uint64(id.LeastSigBits), Partitions: partitions, } - res, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, + res, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.MostSigBits], requestID, pb.BaseCommand_ADD_PARTITION_TO_TXN, cmdAddPartitions) tc.semaphore.Release() if err != nil { @@ -163,11 +163,11 @@ func (tc *transactionCoordinatorClient) addSubscriptionToTxn(id *TxnID, topic st } cmdAddSubscription := &pb.CommandAddSubscriptionToTxn{ RequestId: proto.Uint64(requestID), - TxnidMostBits: proto.Uint64(id.mostSigBits), - TxnidLeastBits: proto.Uint64(id.leastSigBits), + TxnidMostBits: proto.Uint64(id.MostSigBits), + TxnidLeastBits: proto.Uint64(id.LeastSigBits), Subscription: []*pb.Subscription{sub}, } - res, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, + res, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.MostSigBits], requestID, pb.BaseCommand_ADD_SUBSCRIPTION_TO_TXN, cmdAddSubscription) tc.semaphore.Release() if err != nil { @@ -187,10 +187,10 @@ func (tc *transactionCoordinatorClient) endTxn(id *TxnID, action pb.TxnAction) e cmdEndTxn := &pb.CommandEndTxn{ RequestId: proto.Uint64(requestID), TxnAction: &action, - TxnidMostBits: proto.Uint64(id.mostSigBits), - TxnidLeastBits: proto.Uint64(id.leastSigBits), + TxnidMostBits: proto.Uint64(id.MostSigBits), + TxnidLeastBits: proto.Uint64(id.LeastSigBits), } - res, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.mostSigBits], requestID, pb.BaseCommand_END_TXN, cmdEndTxn) + res, err := tc.client.rpcClient.RequestOnCnx(tc.cons[id.MostSigBits], requestID, pb.BaseCommand_END_TXN, cmdEndTxn) tc.semaphore.Release() if err != nil { return err diff --git a/pulsar/transaction_impl.go b/pulsar/transaction_impl.go index 7cc93eca0a..8a24c46429 100644 --- a/pulsar/transaction_impl.go +++ b/pulsar/transaction_impl.go @@ -231,6 +231,8 @@ func (state TxnState) string() string { return "TxnAborted" case TxnTimeout: return "TxnTimeout" + case TxnError: + return "TxnError" default: return "Unknown" } diff --git a/pulsar/transaction_test.go b/pulsar/transaction_test.go index 362e4d2eb6..66a82ccad1 100644 --- a/pulsar/transaction_test.go +++ b/pulsar/transaction_test.go @@ -103,8 +103,8 @@ func TestTxnImplCommitOrAbort(t *testing.T) { //The operations of committing txn1 should success at the first time and fail at the second time. txn1 := createTxn(tc, t) err := txn1.Commit(context.Background()) - require.Nil(t, err, fmt.Sprintf("Failed to commit the transaction %d:%d\n", txn1.txnID.mostSigBits, - txn1.txnID.leastSigBits)) + require.Nil(t, err, fmt.Sprintf("Failed to commit the transaction %d:%d\n", txn1.txnID.MostSigBits, + txn1.txnID.LeastSigBits)) txn1.state = TxnOpen txn1.opsFlow <- true err = txn1.Commit(context.Background()) @@ -117,7 +117,7 @@ func TestTxnImplCommitOrAbort(t *testing.T) { txn2 := newTransaction(*id2, tc, time.Hour) err = txn2.Abort(context.Background()) require.Nil(t, err, fmt.Sprintf("Failed to abort the transaction %d:%d\n", - id2.mostSigBits, id2.leastSigBits)) + id2.MostSigBits, id2.LeastSigBits)) txn2.state = TxnOpen txn2.opsFlow <- true err = txn2.Abort(context.Background()) @@ -209,11 +209,237 @@ func createTcClient(t *testing.T) (*transactionCoordinatorClient, *client) { URL: webServiceURLTLS, TLSTrustCertsFilePath: caCertsPath, Authentication: NewAuthenticationTLS(tlsClientCertPath, tlsClientKeyPath), + EnableTransaction: true, }) require.Nil(t, err, "Failed to create client.") - tcClient := newTransactionCoordinatorClientImpl(c.(*client)) - err = tcClient.start() - require.Nil(t, err, "Failed to start transaction coordinator.") - return tcClient, c.(*client) + return c.(*client).tcClient, c.(*client) +} + +// TestConsumeAndProduceWithTxn is a test function that validates the behavior of producing and consuming +// messages with and without transactions. It consists of the following steps: +// +// 1. Prepare: Create a PulsarClient and initialize the transaction coordinator client. +// 2. Prepare: Create a topic and a subscription. +// 3. Produce 10 messages with a transaction and 10 messages without a transaction. +// - Expectation: The consumer should be able to receive the 10 messages sent without a transaction, +// but not the 10 messages sent with the transaction. +// 4. Commit the transaction and receive the remaining 10 messages. +// - Expectation: The consumer should be able to receive the 10 messages sent with the transaction. +// 5. Clean up: Close the consumer and producer instances. +// +// The test ensures that the consumer can only receive messages sent with a transaction after it is committed, +// and that it can always receive messages sent without a transaction. +func TestConsumeAndProduceWithTxn(t *testing.T) { + // Step 1: Prepare - Create PulsarClient and initialize the transaction coordinator client. + topic := newTopicName() + sub := "my-sub" + _, client := createTcClient(t) + // Step 2: Prepare - Create Topic and Subscription. + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: sub, + }) + assert.NoError(t, err) + producer, _ := client.CreateProducer(ProducerOptions{ + Topic: topic, + SendTimeout: 0, + }) + // Step 3: Open a transaction, send 10 messages with the transaction and 10 messages without the transaction. + // Expectation: We can receive the 10 messages sent without a transaction and + // cannot receive the 10 messages sent with the transaction. + txn, err := client.NewTransaction(time.Hour) + require.Nil(t, err) + for i := 0; i < 10; i++ { + _, err = producer.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }) + require.Nil(t, err) + } + for i := 0; i < 10; i++ { + _, err := producer.Send(context.Background(), &ProducerMessage{ + Transaction: txn, + Payload: make([]byte, 1024), + }) + require.Nil(t, err) + } + // Attempt to receive and acknowledge the 10 messages sent without a transaction. + for i := 0; i < 10; i++ { + msg, _ := consumer.Receive(context.Background()) + assert.NotNil(t, msg) + err = consumer.Ack(msg) + assert.Nil(t, err) + } + // Create a goroutine to attempt receiving a message and send it to the 'done1' channel. + done1 := make(chan Message) + go func() { + msg, _ := consumer.Receive(context.Background()) + err := consumer.AckID(msg.ID()) + require.Nil(t, err) + close(done1) + }() + // Expectation: The consumer should not receive uncommitted messages. + select { + case <-done1: + require.Fail(t, "The consumer should not receive uncommitted message") + case <-time.After(time.Second): + } + // Step 4: After committing the transaction, we should be able to receive the remaining 10 messages. + // Acknowledge the rest of the 10 messages with the transaction. + // Expectation: After committing the transaction, all messages of the subscription will be acknowledged. + _ = txn.Commit(context.Background()) + txn, err = client.NewTransaction(time.Hour) + require.Nil(t, err) + for i := 0; i < 9; i++ { + msg, _ := consumer.Receive(context.Background()) + require.NotNil(t, msg) + err = consumer.AckWithTxn(msg, txn) + require.Nil(t, err) + } + _ = txn.Commit(context.Background()) + <-done1 + consumer.Close() + consumer, _ = client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: sub, + }) + // Create a goroutine to attempt receiving a message and send it to the 'done1' channel. + done2 := make(chan Message) + go func() { + consumer.Receive(context.Background()) + close(done2) + }() + + // Expectation: The consumer should not receive uncommitted messages. + select { + case <-done2: + require.Fail(t, "The consumer should not receive messages") + case <-time.After(time.Second): + } + + // Step 5: Clean up - Close the consumer and producer instances. + consumer.Close() + producer.Close() +} + +func TestAckAndSendWithTxn(t *testing.T) { + // Prepare: Create PulsarClient and initialize the transaction coordinator client. + sourceTopic := newTopicName() + sinkTopic := newTopicName() + sub := "my-sub" + _, client := createTcClient(t) + + // Prepare: Create source and sink topics and subscriptions. + sourceConsumer, _ := client.Subscribe(ConsumerOptions{ + Topic: sourceTopic, + SubscriptionName: sub, + }) + sourceProducer, _ := client.CreateProducer(ProducerOptions{ + Topic: sourceTopic, + SendTimeout: 0, + }) + sinkConsumer, _ := client.Subscribe(ConsumerOptions{ + Topic: sinkTopic, + SubscriptionName: sub, + }) + sinkProducer, _ := client.CreateProducer(ProducerOptions{ + Topic: sinkTopic, + SendTimeout: 0, + }) + + // Produce 10 messages to the source topic. + for i := 0; i < 10; i++ { + _, err := sourceProducer.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }) + require.Nil(t, err) + } + + // Open a transaction and consume messages from the source topic while sending messages to the sink topic. + txn, err := client.NewTransaction(time.Hour) + require.Nil(t, err) + + for i := 0; i < 10; i++ { + msg, _ := sourceConsumer.Receive(context.Background()) + require.NotNil(t, msg) + + payload := msg.Payload() + _, err := sinkProducer.Send(context.Background(), &ProducerMessage{ + Transaction: txn, + Payload: payload, + }) + require.Nil(t, err) + + err = sourceConsumer.AckWithTxn(msg, txn) + require.Nil(t, err) + } + + // Commit the transaction. + _ = txn.Commit(context.Background()) + + // Verify that the messages are available in the sink topic. + for i := 0; i < 10; i++ { + msg, _ := sinkConsumer.Receive(context.Background()) + require.NotNil(t, msg) + err = sinkConsumer.Ack(msg) + require.Nil(t, err) + } + + // Clean up: Close consumers and producers. + sourceConsumer.Close() + sourceProducer.Close() + sinkConsumer.Close() + sinkProducer.Close() +} + +func TestTransactionAbort(t *testing.T) { + // Prepare: Create PulsarClient and initialize the transaction coordinator client. + topic := newTopicName() + sub := "my-sub" + _, client := createTcClient(t) + + // Prepare: Create Topic and Subscription. + consumer, _ := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: sub, + }) + producer, _ := client.CreateProducer(ProducerOptions{ + Topic: topic, + SendTimeout: 0, + }) + + // Open a transaction and send 10 messages with the transaction. + txn, err := client.NewTransaction(time.Hour) + require.Nil(t, err) + + for i := 0; i < 10; i++ { + _, err = producer.Send(context.Background(), &ProducerMessage{ + Transaction: txn, + Payload: make([]byte, 1024), + }) + require.Nil(t, err) + } + + // Abort the transaction. + _ = txn.Abort(context.Background()) + + // Expectation: The consumer should not receive any messages. + done := make(chan struct{}) + go func() { + _, err := consumer.Receive(context.Background()) + if err != nil { + close(done) + } + }() + + select { + case <-done: + // The consumer should not receive any messages. + require.Fail(t, "The consumer should not receive any messages") + case <-time.After(time.Second): + } + + // Clean up: Close the consumer and producer instances. + consumer.Close() + producer.Close() } From d51e2920e705852143be408083541855c52b222e Mon Sep 17 00:00:00 2001 From: Jun Ma <60642177+momo-jun@users.noreply.github.com> Date: Wed, 10 May 2023 16:17:23 +0800 Subject: [PATCH 048/348] [Doc] Add links to client docs and feature matrix in README.md (#1014) ### Motivation The new [client feature matrix](https://pulsar.apache.org/client-feature-matrix/) page has been live, so this PR makes the required updates on the doc side. ### Modifications Add a paragraph to notify both users and contributors about the new client feature matrix. --- README.md | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index 64700f8797..dee379cfb1 100644 --- a/README.md +++ b/README.md @@ -18,13 +18,15 @@ under the License. --> + [![PkgGoDev](https://pkg.go.dev/badge/github.com/apache/pulsar-client-go)](https://pkg.go.dev/github.com/apache/pulsar-client-go) [![Go Report Card](https://goreportcard.com/badge/github.com/apache/pulsar-client-go)](https://goreportcard.com/report/github.com/apache/pulsar-client-go) [![Language](https://img.shields.io/badge/Language-Go-blue.svg)](https://golang.org/) [![LICENSE](https://img.shields.io/hexpm/l/pulsar.svg)](/~https://github.com/apache/pulsar-client-go/blob/master/LICENSE) + # Apache Pulsar Go Client Library -A Go client library for [Apache Pulsar](https://pulsar.apache.org/). +A Go client library for [Apache Pulsar](https://pulsar.apache.org/). For the supported Pulsar features, see [Client Feature Matrix](https://pulsar.apache.org/client-feature-matrix/). ## Purpose @@ -32,7 +34,7 @@ This project is a pure-Go client library for Pulsar that does not depend on the C++ Pulsar library. Once feature parity and stability are reached, this will supersede the current -CGo based library. +CGo-based library. ## Requirements @@ -156,12 +158,14 @@ Run the tests with specific versions of GOLANG and PULSAR: Contributions are welcomed and greatly appreciated. See [CONTRIBUTING.md](CONTRIBUTING.md) for details on submitting patches and the contribution workflow. +If your contribution adds Pulsar features for Go clients, you need to update both the [Pulsar docs](https://pulsar.apache.org/docs/client-libraries/) and the [Client Feature Matrix](https://pulsar.apache.org/client-feature-matrix/). See [Contribution Guide](https://pulsar.apache.org/contribute/site-intro/#pages) for more details. + ## Community ##### Mailing lists | Name | Scope | | | | -|:----------------------------------------------------------|:--------------------------------|:------------------------------------------------------|:----------------------------------------------------------|:-------------------------------------------------------------------| +| :-------------------------------------------------------- | :------------------------------ | :---------------------------------------------------- | :-------------------------------------------------------- | :----------------------------------------------------------------- | | [users@pulsar.apache.org](mailto:users@pulsar.apache.org) | User-related discussions | [Subscribe](mailto:users-subscribe@pulsar.apache.org) | [Unsubscribe](mailto:users-unsubscribe@pulsar.apache.org) | [Archives](http://mail-archives.apache.org/mod_mbox/pulsar-users/) | | [dev@pulsar.apache.org](mailto:dev@pulsar.apache.org) | Development-related discussions | [Subscribe](mailto:dev-subscribe@pulsar.apache.org) | [Unsubscribe](mailto:dev-unsubscribe@pulsar.apache.org) | [Archives](http://mail-archives.apache.org/mod_mbox/pulsar-dev/) | @@ -179,13 +183,13 @@ Licensed under the Apache License, Version 2.0: http://www.apache.org/licenses/L ### Go module 'ambiguous import' error -If you've upgraded from a previous version of this library, you may run into an 'ambigous import' error when building. +If you've upgraded from a previous version of this library, you may run into an 'ambiguous import' error when building. ``` github.com/apache/pulsar-client-go/oauth2: ambiguous import: found package github.com/apache/pulsar-client-go/oauth2 in multiple modules ``` -The fix for this is to make sure you don't have any references in your `go.mod` file to the old oauth2 module path. So remove any lines +The fix for this is to make sure you don't have any references in your `go.mod` file to the old oauth2 module path. So remove any lines similar to the following, and then run `go mod tidy`. ``` From 3367cc0cf877faadce0671c412ec827af7d27a16 Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Thu, 11 May 2023 17:25:12 +0800 Subject: [PATCH 049/348] [fix][Flaky Test] Fix flaky test TestMaxPendingChunkMessages() (#1003) ### Motivation Old TestMaxPendingChunkMessages() uses the concurrent message publish to make the consumer discard unavailable chunk. And it's flaky. So the `sendSingeChunk()` is introduced to manual create scenarios where old chunks should be discarded. ### Modifications - Fix `TestMaxPendingChunkMessages()` --- pulsar/message_chunking_test.go | 135 ++++++++++++++++---------------- 1 file changed, 67 insertions(+), 68 deletions(-) diff --git a/pulsar/message_chunking_test.go b/pulsar/message_chunking_test.go index 0e1214d311..ee3ab17760 100644 --- a/pulsar/message_chunking_test.go +++ b/pulsar/message_chunking_test.go @@ -24,10 +24,13 @@ import ( "math/rand" "net/http" "strings" - "sync" "testing" "time" + "github.com/apache/pulsar-client-go/pulsar/internal" + + "google.golang.org/protobuf/proto" + "github.com/stretchr/testify/assert" ) @@ -148,8 +151,6 @@ func TestLargeMessage(t *testing.T) { } func TestMaxPendingChunkMessages(t *testing.T) { - rand.Seed(time.Now().Unix()) - client, err := NewClient(ClientOptions{ URL: lookupURL, }) @@ -157,83 +158,52 @@ func TestMaxPendingChunkMessages(t *testing.T) { defer client.Close() topic := newTopicName() + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + DisableBatching: true, + EnableChunking: true, + ChunkMaxMessageSize: 10, + }) + assert.NoError(t, err) + assert.NotNil(t, producer) - totalProducers := 5 - producers := make([]Producer, 0, 20) - defer func() { - for _, p := range producers { - p.Close() - } - }() - - clients := make([]Client, 0, 20) - defer func() { - for _, c := range clients { - c.Close() - } - }() - - for j := 0; j < totalProducers; j++ { - pc, err := NewClient(ClientOptions{ - URL: lookupURL, - }) - assert.Nil(t, err) - clients = append(clients, pc) - producer, err := pc.CreateProducer(ProducerOptions{ - Topic: topic, - DisableBatching: true, - EnableChunking: true, - ChunkMaxMessageSize: 10, - }) - assert.NoError(t, err) - assert.NotNil(t, producer) - producers = append(producers, producer) - } - - consumer, err := client.Subscribe(ConsumerOptions{ + c, err := client.Subscribe(ConsumerOptions{ Topic: topic, Type: Exclusive, SubscriptionName: "chunk-subscriber", MaxPendingChunkedMessage: 1, }) assert.NoError(t, err) - assert.NotNil(t, consumer) - defer consumer.Close() + assert.NotNil(t, c) + defer c.Close() + pc := c.(*consumer).consumers[0] - totalMsgs := 40 - wg := sync.WaitGroup{} - wg.Add(totalMsgs * totalProducers) - for i := 0; i < totalMsgs; i++ { - for j := 0; j < totalProducers; j++ { - p := producers[j] - go func() { - ID, err := p.Send(context.Background(), &ProducerMessage{ - Payload: createTestMessagePayload(50), - }) - assert.NoError(t, err) - assert.NotNil(t, ID) - wg.Done() - }() - } - } - wg.Wait() + sendSingleChunk(producer, "0", 0, 2) + // MaxPendingChunkedMessage is 1, the chunked message with uuid 0 will be discarded + sendSingleChunk(producer, "1", 0, 2) - received := 0 - for i := 0; i < totalMsgs*totalProducers; i++ { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) - msg, err := consumer.Receive(ctx) - cancel() - if msg == nil || (err != nil && errors.Is(err, context.DeadlineExceeded)) { - break - } + // chunkedMsgCtx with uuid 0 should be discarded + retryAssert(t, 3, 200, func() {}, func(t assert.TestingT) bool { + pc.chunkedMsgCtxMap.mu.Lock() + defer pc.chunkedMsgCtxMap.mu.Unlock() + return assert.Equal(t, 1, len(pc.chunkedMsgCtxMap.chunkedMsgCtxs)) + }) - received++ + sendSingleChunk(producer, "1", 1, 2) - err = consumer.Ack(msg) - assert.NoError(t, err) - } + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + msg, err := c.Receive(ctx) + cancel() - assert.NotEqual(t, totalMsgs*totalProducers, received) + assert.NoError(t, err) + assert.Equal(t, "chunk-1-0|chunk-1-1|", string(msg.Payload())) + + // Ensure that the chunked message of uuid 0 is discarded. + sendSingleChunk(producer, "0", 1, 2) + ctx, cancel = context.WithTimeout(context.Background(), 5*time.Second) + msg, err = c.Receive(ctx) + cancel() + assert.True(t, errors.Is(err, context.DeadlineExceeded)) } func TestExpireIncompleteChunks(t *testing.T) { @@ -576,3 +546,32 @@ func createTestMessagePayload(size int) []byte { } return payload } + +//nolint:all +func sendSingleChunk(p Producer, uuid string, chunkID int, totalChunks int) { + msg := &ProducerMessage{ + Payload: []byte(fmt.Sprintf("chunk-%s-%d|", uuid, chunkID)), + } + producerImpl := p.(*producer).producers[0].(*partitionProducer) + mm := producerImpl.genMetadata(msg, len(msg.Payload), time.Now()) + mm.Uuid = proto.String(uuid) + mm.NumChunksFromMsg = proto.Int32(int32(totalChunks)) + mm.TotalChunkMsgSize = proto.Int32(int32(len(msg.Payload))) + mm.ChunkId = proto.Int32(int32(chunkID)) + producerImpl.updateMetadataSeqID(mm, msg) + + doneCh := make(chan struct{}) + producerImpl.internalSingleSend( + mm, + msg.Payload, + &sendRequest{ + callback: func(id MessageID, producerMessage *ProducerMessage, err error) { + close(doneCh) + }, + msg: msg, + }, + uint32(internal.MaxMessageSize), + ) + + <-doneCh +} From 3f4a18b7e8a12c807769b1f673ac09d7f1c900df Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Fri, 12 May 2023 15:32:27 +0800 Subject: [PATCH 050/348] [fix] Fix flaky test in `negative_acks_tracker_test.go` (#1017) --- pulsar/negative_acks_tracker_test.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pulsar/negative_acks_tracker_test.go b/pulsar/negative_acks_tracker_test.go index 12d33dee22..9493412333 100644 --- a/pulsar/negative_acks_tracker_test.go +++ b/pulsar/negative_acks_tracker_test.go @@ -42,10 +42,11 @@ func newNackMockedConsumer(nackBackoffPolicy NackBackoffPolicy) *nackMockedConsu go func() { // since the client ticks at an interval of delay / 3 // wait another interval to ensure we get all messages + // And we wait another 100 milliseconds to reduce the flaky if nackBackoffPolicy == nil { - time.Sleep(testNackDelay + 101*time.Millisecond) + time.Sleep(testNackDelay + 200*time.Millisecond) } else { - time.Sleep(nackBackoffPolicy.Next(1) + 101*time.Millisecond) + time.Sleep(nackBackoffPolicy.Next(1) + 200*time.Millisecond) } t.lock.Lock() defer t.lock.Unlock() From 27e36250f35094b203483c946bc4406c6119ce62 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 17 May 2023 18:18:20 +0800 Subject: [PATCH 051/348] [fix] Fix event time not being set when batching is disabled (#1015) Fixes #1013 ### Motivation The event time is not set when batching is disabled. The event time will be lost. This is a regression bug in 0.10.0. ### Modifications * Set the event time when sending single message --- pulsar/producer_partition.go | 4 ++++ pulsar/producer_test.go | 33 +++++++++++++++++++++++++++++++++ 2 files changed, 37 insertions(+) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index e77c929c3c..fc67f51581 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -735,6 +735,10 @@ func (p *partitionProducer) genMetadata(msg *ProducerMessage, UncompressedSize: proto.Uint32(uint32(uncompressedSize)), } + if !msg.EventTime.IsZero() { + mm.EventTime = proto.Uint64(internal.TimestampMillis(msg.EventTime)) + } + if msg.Key != "" { mm.PartitionKey = proto.String(msg.Key) } diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index a6f5e3972c..b587975c9c 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -1905,3 +1905,36 @@ func TestMemLimitContextCancel(t *testing.T) { }) assert.NoError(t, err) } + +func TestSendMessagesWithMetadata(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + + topic := newTopicName() + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + DisableBatching: true, + }) + assert.Nil(t, err) + + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + }) + assert.Nil(t, err) + + msg := &ProducerMessage{EventTime: time.Now().Local(), + Payload: []byte("msg")} + + _, err = producer.Send(context.Background(), msg) + assert.Nil(t, err) + + recvMsg, err := consumer.Receive(context.Background()) + assert.Nil(t, err) + + assert.Equal(t, internal.TimestampMillis(recvMsg.EventTime()), internal.TimestampMillis(msg.EventTime)) +} From 2eb340511dbf0602f7c561a1d86e179098f4662b Mon Sep 17 00:00:00 2001 From: Benjamin Pereto Date: Wed, 31 May 2023 08:53:45 +0200 Subject: [PATCH 052/348] fix: use maphash instead of crypto/sha256 for hash function of hashmap in Schema.hash() (#1022) - replace sha256 hash function with hash/maphash - use uint64 (expected from maphash) as schema cache hashmap key - init static seed as it is random generated --- pulsar/producer_partition.go | 8 ++++---- pulsar/schema.go | 12 +++++++----- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index fc67f51581..6c2547bb9e 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -101,12 +101,12 @@ type partitionProducer struct { type schemaCache struct { lock sync.RWMutex - schemas map[string][]byte + schemas map[uint64][]byte } func newSchemaCache() *schemaCache { return &schemaCache{ - schemas: make(map[string][]byte), + schemas: make(map[uint64][]byte), } } @@ -122,9 +122,9 @@ func (s *schemaCache) Get(schema *SchemaInfo) (schemaVersion []byte) { s.lock.RLock() defer s.lock.RUnlock() - key := schema.hash() - return s.schemas[key] + return s.schemas[schema.hash()] } + func newPartitionProducer(client *client, topic string, options *ProducerOptions, partitionIdx int, metrics *internal.LeveledMetrics) ( *partitionProducer, error) { diff --git a/pulsar/schema.go b/pulsar/schema.go index 405049dcee..0b413d40a6 100644 --- a/pulsar/schema.go +++ b/pulsar/schema.go @@ -19,10 +19,9 @@ package pulsar import ( "bytes" - "crypto/sha256" - "encoding/hex" "encoding/json" "fmt" + "hash/maphash" "reflect" "unsafe" @@ -69,10 +68,11 @@ type SchemaInfo struct { Properties map[string]string } -func (s SchemaInfo) hash() string { - h := sha256.New() +func (s SchemaInfo) hash() uint64 { + h := maphash.Hash{} + h.SetSeed(seed) h.Write([]byte(s.Schema)) - return hex.EncodeToString(h.Sum(nil)) + return h.Sum64() } type Schema interface { @@ -183,6 +183,8 @@ type ProtoSchema struct { SchemaInfo } +var seed = maphash.MakeSeed() + // NewProtoSchema creates a new ProtoSchema // Note: the function will panic if creation of codec fails func NewProtoSchema(protoAvroSchemaDef string, properties map[string]string) *ProtoSchema { From 4c8246c78c0678a72b57f3e171c1a4e6cff31450 Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 7 Jun 2023 23:14:58 +0800 Subject: [PATCH 053/348] chore: improve logs on failTimeoutMessages (#1025) --- pulsar/producer_partition.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 6c2547bb9e..c4a460eed4 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -940,7 +940,7 @@ func (p *partitionProducer) failTimeoutMessages() { t.Reset(p.options.SendTimeout) continue } - p.log.Infof("Failing %d messages", viewSize) + p.log.Infof("Failing %d messages on timeout %s", viewSize, p.options.SendTimeout) lastViewItem := curViewItems[viewSize-1].(*pendingItem) // iterate at most viewSize items From 811030457a4fff8c8ea682ffbe82ccc28a8e184d Mon Sep 17 00:00:00 2001 From: tison Date: Mon, 12 Jun 2023 20:25:17 +0800 Subject: [PATCH 054/348] fix: Delete LICENSE-go-rate.txt (#1028) This fixes /~https://github.com/apache/pulsar-client-go/issues/1023. This is a follow-up to #952. --- distribution/license/LICENSE-go-rate.txt | 675 ----------------------- 1 file changed, 675 deletions(-) delete mode 100644 distribution/license/LICENSE-go-rate.txt diff --git a/distribution/license/LICENSE-go-rate.txt b/distribution/license/LICENSE-go-rate.txt deleted file mode 100644 index 13917f00d7..0000000000 --- a/distribution/license/LICENSE-go-rate.txt +++ /dev/null @@ -1,675 +0,0 @@ - - GNU GENERAL PUBLIC LICENSE - Version 3, 29 June 2007 - - Copyright (C) 2007 Free Software Foundation, Inc. - Everyone is permitted to copy and distribute verbatim copies - of this license document, but changing it is not allowed. - - Preamble - - The GNU General Public License is a free, copyleft license for -software and other kinds of works. - - The licenses for most software and other practical works are designed -to take away your freedom to share and change the works. By contrast, -the GNU General Public License is intended to guarantee your freedom to -share and change all versions of a program--to make sure it remains free -software for all its users. We, the Free Software Foundation, use the -GNU General Public License for most of our software; it applies also to -any other work released this way by its authors. You can apply it to -your programs, too. - - When we speak of free software, we are referring to freedom, not -price. Our General Public Licenses are designed to make sure that you -have the freedom to distribute copies of free software (and charge for -them if you wish), that you receive source code or can get it if you -want it, that you can change the software or use pieces of it in new -free programs, and that you know you can do these things. - - To protect your rights, we need to prevent others from denying you -these rights or asking you to surrender the rights. Therefore, you have -certain responsibilities if you distribute copies of the software, or if -you modify it: responsibilities to respect the freedom of others. - - For example, if you distribute copies of such a program, whether -gratis or for a fee, you must pass on to the recipients the same -freedoms that you received. You must make sure that they, too, receive -or can get the source code. And you must show them these terms so they -know their rights. - - Developers that use the GNU GPL protect your rights with two steps: -(1) assert copyright on the software, and (2) offer you this License -giving you legal permission to copy, distribute and/or modify it. - - For the developers' and authors' protection, the GPL clearly explains -that there is no warranty for this free software. For both users' and -authors' sake, the GPL requires that modified versions be marked as -changed, so that their problems will not be attributed erroneously to -authors of previous versions. - - Some devices are designed to deny users access to install or run -modified versions of the software inside them, although the manufacturer -can do so. This is fundamentally incompatible with the aim of -protecting users' freedom to change the software. The systematic -pattern of such abuse occurs in the area of products for individuals to -use, which is precisely where it is most unacceptable. Therefore, we -have designed this version of the GPL to prohibit the practice for those -products. If such problems arise substantially in other domains, we -stand ready to extend this provision to those domains in future versions -of the GPL, as needed to protect the freedom of users. - - Finally, every program is threatened constantly by software patents. -States should not allow patents to restrict development and use of -software on general-purpose computers, but in those that do, we wish to -avoid the special danger that patents applied to a free program could -make it effectively proprietary. To prevent this, the GPL assures that -patents cannot be used to render the program non-free. - - The precise terms and conditions for copying, distribution and -modification follow. - - TERMS AND CONDITIONS - - 0. Definitions. - - "This License" refers to version 3 of the GNU General Public License. - - "Copyright" also means copyright-like laws that apply to other kinds of -works, such as semiconductor masks. - - "The Program" refers to any copyrightable work licensed under this -License. Each licensee is addressed as "you". "Licensees" and -"recipients" may be individuals or organizations. - - To "modify" a work means to copy from or adapt all or part of the work -in a fashion requiring copyright permission, other than the making of an -exact copy. The resulting work is called a "modified version" of the -earlier work or a work "based on" the earlier work. - - A "covered work" means either the unmodified Program or a work based -on the Program. - - To "propagate" a work means to do anything with it that, without -permission, would make you directly or secondarily liable for -infringement under applicable copyright law, except executing it on a -computer or modifying a private copy. Propagation includes copying, -distribution (with or without modification), making available to the -public, and in some countries other activities as well. - - To "convey" a work means any kind of propagation that enables other -parties to make or receive copies. Mere interaction with a user through -a computer network, with no transfer of a copy, is not conveying. - - An interactive user interface displays "Appropriate Legal Notices" -to the extent that it includes a convenient and prominently visible -feature that (1) displays an appropriate copyright notice, and (2) -tells the user that there is no warranty for the work (except to the -extent that warranties are provided), that licensees may convey the -work under this License, and how to view a copy of this License. If -the interface presents a list of user commands or options, such as a -menu, a prominent item in the list meets this criterion. - - 1. Source Code. - - The "source code" for a work means the preferred form of the work -for making modifications to it. "Object code" means any non-source -form of a work. - - A "Standard Interface" means an interface that either is an official -standard defined by a recognized standards body, or, in the case of -interfaces specified for a particular programming language, one that -is widely used among developers working in that language. - - The "System Libraries" of an executable work include anything, other -than the work as a whole, that (a) is included in the normal form of -packaging a Major Component, but which is not part of that Major -Component, and (b) serves only to enable use of the work with that -Major Component, or to implement a Standard Interface for which an -implementation is available to the public in source code form. A -"Major Component", in this context, means a major essential component -(kernel, window system, and so on) of the specific operating system -(if any) on which the executable work runs, or a compiler used to -produce the work, or an object code interpreter used to run it. - - The "Corresponding Source" for a work in object code form means all -the source code needed to generate, install, and (for an executable -work) run the object code and to modify the work, including scripts to -control those activities. However, it does not include the work's -System Libraries, or general-purpose tools or generally available free -programs which are used unmodified in performing those activities but -which are not part of the work. For example, Corresponding Source -includes interface definition files associated with source files for -the work, and the source code for shared libraries and dynamically -linked subprograms that the work is specifically designed to require, -such as by intimate data communication or control flow between those -subprograms and other parts of the work. - - The Corresponding Source need not include anything that users -can regenerate automatically from other parts of the Corresponding -Source. - - The Corresponding Source for a work in source code form is that -same work. - - 2. Basic Permissions. - - All rights granted under this License are granted for the term of -copyright on the Program, and are irrevocable provided the stated -conditions are met. This License explicitly affirms your unlimited -permission to run the unmodified Program. The output from running a -covered work is covered by this License only if the output, given its -content, constitutes a covered work. This License acknowledges your -rights of fair use or other equivalent, as provided by copyright law. - - You may make, run and propagate covered works that you do not -convey, without conditions so long as your license otherwise remains -in force. You may convey covered works to others for the sole purpose -of having them make modifications exclusively for you, or provide you -with facilities for running those works, provided that you comply with -the terms of this License in conveying all material for which you do -not control copyright. Those thus making or running the covered works -for you must do so exclusively on your behalf, under your direction -and control, on terms that prohibit them from making any copies of -your copyrighted material outside their relationship with you. - - Conveying under any other circumstances is permitted solely under -the conditions stated below. Sublicensing is not allowed; section 10 -makes it unnecessary. - - 3. Protecting Users' Legal Rights From Anti-Circumvention Law. - - No covered work shall be deemed part of an effective technological -measure under any applicable law fulfilling obligations under article -11 of the WIPO copyright treaty adopted on 20 December 1996, or -similar laws prohibiting or restricting circumvention of such -measures. - - When you convey a covered work, you waive any legal power to forbid -circumvention of technological measures to the extent such circumvention -is effected by exercising rights under this License with respect to -the covered work, and you disclaim any intention to limit operation or -modification of the work as a means of enforcing, against the work's -users, your or third parties' legal rights to forbid circumvention of -technological measures. - - 4. Conveying Verbatim Copies. - - You may convey verbatim copies of the Program's source code as you -receive it, in any medium, provided that you conspicuously and -appropriately publish on each copy an appropriate copyright notice; -keep intact all notices stating that this License and any -non-permissive terms added in accord with section 7 apply to the code; -keep intact all notices of the absence of any warranty; and give all -recipients a copy of this License along with the Program. - - You may charge any price or no price for each copy that you convey, -and you may offer support or warranty protection for a fee. - - 5. Conveying Modified Source Versions. - - You may convey a work based on the Program, or the modifications to -produce it from the Program, in the form of source code under the -terms of section 4, provided that you also meet all of these conditions: - - a) The work must carry prominent notices stating that you modified - it, and giving a relevant date. - - b) The work must carry prominent notices stating that it is - released under this License and any conditions added under section - 7. This requirement modifies the requirement in section 4 to - "keep intact all notices". - - c) You must license the entire work, as a whole, under this - License to anyone who comes into possession of a copy. This - License will therefore apply, along with any applicable section 7 - additional terms, to the whole of the work, and all its parts, - regardless of how they are packaged. This License gives no - permission to license the work in any other way, but it does not - invalidate such permission if you have separately received it. - - d) If the work has interactive user interfaces, each must display - Appropriate Legal Notices; however, if the Program has interactive - interfaces that do not display Appropriate Legal Notices, your - work need not make them do so. - - A compilation of a covered work with other separate and independent -works, which are not by their nature extensions of the covered work, -and which are not combined with it such as to form a larger program, -in or on a volume of a storage or distribution medium, is called an -"aggregate" if the compilation and its resulting copyright are not -used to limit the access or legal rights of the compilation's users -beyond what the individual works permit. Inclusion of a covered work -in an aggregate does not cause this License to apply to the other -parts of the aggregate. - - 6. Conveying Non-Source Forms. - - You may convey a covered work in object code form under the terms -of sections 4 and 5, provided that you also convey the -machine-readable Corresponding Source under the terms of this License, -in one of these ways: - - a) Convey the object code in, or embodied in, a physical product - (including a physical distribution medium), accompanied by the - Corresponding Source fixed on a durable physical medium - customarily used for software interchange. - - b) Convey the object code in, or embodied in, a physical product - (including a physical distribution medium), accompanied by a - written offer, valid for at least three years and valid for as - long as you offer spare parts or customer support for that product - model, to give anyone who possesses the object code either (1) a - copy of the Corresponding Source for all the software in the - product that is covered by this License, on a durable physical - medium customarily used for software interchange, for a price no - more than your reasonable cost of physically performing this - conveying of source, or (2) access to copy the - Corresponding Source from a network server at no charge. - - c) Convey individual copies of the object code with a copy of the - written offer to provide the Corresponding Source. This - alternative is allowed only occasionally and noncommercially, and - only if you received the object code with such an offer, in accord - with subsection 6b. - - d) Convey the object code by offering access from a designated - place (gratis or for a charge), and offer equivalent access to the - Corresponding Source in the same way through the same place at no - further charge. You need not require recipients to copy the - Corresponding Source along with the object code. If the place to - copy the object code is a network server, the Corresponding Source - may be on a different server (operated by you or a third party) - that supports equivalent copying facilities, provided you maintain - clear directions next to the object code saying where to find the - Corresponding Source. Regardless of what server hosts the - Corresponding Source, you remain obligated to ensure that it is - available for as long as needed to satisfy these requirements. - - e) Convey the object code using peer-to-peer transmission, provided - you inform other peers where the object code and Corresponding - Source of the work are being offered to the general public at no - charge under subsection 6d. - - A separable portion of the object code, whose source code is excluded -from the Corresponding Source as a System Library, need not be -included in conveying the object code work. - - A "User Product" is either (1) a "consumer product", which means any -tangible personal property which is normally used for personal, family, -or household purposes, or (2) anything designed or sold for incorporation -into a dwelling. In determining whether a product is a consumer product, -doubtful cases shall be resolved in favor of coverage. For a particular -product received by a particular user, "normally used" refers to a -typical or common use of that class of product, regardless of the status -of the particular user or of the way in which the particular user -actually uses, or expects or is expected to use, the product. A product -is a consumer product regardless of whether the product has substantial -commercial, industrial or non-consumer uses, unless such uses represent -the only significant mode of use of the product. - - "Installation Information" for a User Product means any methods, -procedures, authorization keys, or other information required to install -and execute modified versions of a covered work in that User Product from -a modified version of its Corresponding Source. The information must -suffice to ensure that the continued functioning of the modified object -code is in no case prevented or interfered with solely because -modification has been made. - - If you convey an object code work under this section in, or with, or -specifically for use in, a User Product, and the conveying occurs as -part of a transaction in which the right of possession and use of the -User Product is transferred to the recipient in perpetuity or for a -fixed term (regardless of how the transaction is characterized), the -Corresponding Source conveyed under this section must be accompanied -by the Installation Information. But this requirement does not apply -if neither you nor any third party retains the ability to install -modified object code on the User Product (for example, the work has -been installed in ROM). - - The requirement to provide Installation Information does not include a -requirement to continue to provide support service, warranty, or updates -for a work that has been modified or installed by the recipient, or for -the User Product in which it has been modified or installed. Access to a -network may be denied when the modification itself materially and -adversely affects the operation of the network or violates the rules and -protocols for communication across the network. - - Corresponding Source conveyed, and Installation Information provided, -in accord with this section must be in a format that is publicly -documented (and with an implementation available to the public in -source code form), and must require no special password or key for -unpacking, reading or copying. - - 7. Additional Terms. - - "Additional permissions" are terms that supplement the terms of this -License by making exceptions from one or more of its conditions. -Additional permissions that are applicable to the entire Program shall -be treated as though they were included in this License, to the extent -that they are valid under applicable law. If additional permissions -apply only to part of the Program, that part may be used separately -under those permissions, but the entire Program remains governed by -this License without regard to the additional permissions. - - When you convey a copy of a covered work, you may at your option -remove any additional permissions from that copy, or from any part of -it. (Additional permissions may be written to require their own -removal in certain cases when you modify the work.) You may place -additional permissions on material, added by you to a covered work, -for which you have or can give appropriate copyright permission. - - Notwithstanding any other provision of this License, for material you -add to a covered work, you may (if authorized by the copyright holders of -that material) supplement the terms of this License with terms: - - a) Disclaiming warranty or limiting liability differently from the - terms of sections 15 and 16 of this License; or - - b) Requiring preservation of specified reasonable legal notices or - author attributions in that material or in the Appropriate Legal - Notices displayed by works containing it; or - - c) Prohibiting misrepresentation of the origin of that material, or - requiring that modified versions of such material be marked in - reasonable ways as different from the original version; or - - d) Limiting the use for publicity purposes of names of licensors or - authors of the material; or - - e) Declining to grant rights under trademark law for use of some - trade names, trademarks, or service marks; or - - f) Requiring indemnification of licensors and authors of that - material by anyone who conveys the material (or modified versions of - it) with contractual assumptions of liability to the recipient, for - any liability that these contractual assumptions directly impose on - those licensors and authors. - - All other non-permissive additional terms are considered "further -restrictions" within the meaning of section 10. If the Program as you -received it, or any part of it, contains a notice stating that it is -governed by this License along with a term that is a further -restriction, you may remove that term. If a license document contains -a further restriction but permits relicensing or conveying under this -License, you may add to a covered work material governed by the terms -of that license document, provided that the further restriction does -not survive such relicensing or conveying. - - If you add terms to a covered work in accord with this section, you -must place, in the relevant source files, a statement of the -additional terms that apply to those files, or a notice indicating -where to find the applicable terms. - - Additional terms, permissive or non-permissive, may be stated in the -form of a separately written license, or stated as exceptions; -the above requirements apply either way. - - 8. Termination. - - You may not propagate or modify a covered work except as expressly -provided under this License. Any attempt otherwise to propagate or -modify it is void, and will automatically terminate your rights under -this License (including any patent licenses granted under the third -paragraph of section 11). - - However, if you cease all violation of this License, then your -license from a particular copyright holder is reinstated (a) -provisionally, unless and until the copyright holder explicitly and -finally terminates your license, and (b) permanently, if the copyright -holder fails to notify you of the violation by some reasonable means -prior to 60 days after the cessation. - - Moreover, your license from a particular copyright holder is -reinstated permanently if the copyright holder notifies you of the -violation by some reasonable means, this is the first time you have -received notice of violation of this License (for any work) from that -copyright holder, and you cure the violation prior to 30 days after -your receipt of the notice. - - Termination of your rights under this section does not terminate the -licenses of parties who have received copies or rights from you under -this License. If your rights have been terminated and not permanently -reinstated, you do not qualify to receive new licenses for the same -material under section 10. - - 9. Acceptance Not Required for Having Copies. - - You are not required to accept this License in order to receive or -run a copy of the Program. Ancillary propagation of a covered work -occurring solely as a consequence of using peer-to-peer transmission -to receive a copy likewise does not require acceptance. However, -nothing other than this License grants you permission to propagate or -modify any covered work. These actions infringe copyright if you do -not accept this License. Therefore, by modifying or propagating a -covered work, you indicate your acceptance of this License to do so. - - 10. Automatic Licensing of Downstream Recipients. - - Each time you convey a covered work, the recipient automatically -receives a license from the original licensors, to run, modify and -propagate that work, subject to this License. You are not responsible -for enforcing compliance by third parties with this License. - - An "entity transaction" is a transaction transferring control of an -organization, or substantially all assets of one, or subdividing an -organization, or merging organizations. If propagation of a covered -work results from an entity transaction, each party to that -transaction who receives a copy of the work also receives whatever -licenses to the work the party's predecessor in interest had or could -give under the previous paragraph, plus a right to possession of the -Corresponding Source of the work from the predecessor in interest, if -the predecessor has it or can get it with reasonable efforts. - - You may not impose any further restrictions on the exercise of the -rights granted or affirmed under this License. For example, you may -not impose a license fee, royalty, or other charge for exercise of -rights granted under this License, and you may not initiate litigation -(including a cross-claim or counterclaim in a lawsuit) alleging that -any patent claim is infringed by making, using, selling, offering for -sale, or importing the Program or any portion of it. - - 11. Patents. - - A "contributor" is a copyright holder who authorizes use under this -License of the Program or a work on which the Program is based. The -work thus licensed is called the contributor's "contributor version". - - A contributor's "essential patent claims" are all patent claims -owned or controlled by the contributor, whether already acquired or -hereafter acquired, that would be infringed by some manner, permitted -by this License, of making, using, or selling its contributor version, -but do not include claims that would be infringed only as a -consequence of further modification of the contributor version. For -purposes of this definition, "control" includes the right to grant -patent sublicenses in a manner consistent with the requirements of -this License. - - Each contributor grants you a non-exclusive, worldwide, royalty-free -patent license under the contributor's essential patent claims, to -make, use, sell, offer for sale, import and otherwise run, modify and -propagate the contents of its contributor version. - - In the following three paragraphs, a "patent license" is any express -agreement or commitment, however denominated, not to enforce a patent -(such as an express permission to practice a patent or covenant not to -sue for patent infringement). To "grant" such a patent license to a -party means to make such an agreement or commitment not to enforce a -patent against the party. - - If you convey a covered work, knowingly relying on a patent license, -and the Corresponding Source of the work is not available for anyone -to copy, free of charge and under the terms of this License, through a -publicly available network server or other readily accessible means, -then you must either (1) cause the Corresponding Source to be so -available, or (2) arrange to deprive yourself of the benefit of the -patent license for this particular work, or (3) arrange, in a manner -consistent with the requirements of this License, to extend the patent -license to downstream recipients. "Knowingly relying" means you have -actual knowledge that, but for the patent license, your conveying the -covered work in a country, or your recipient's use of the covered work -in a country, would infringe one or more identifiable patents in that -country that you have reason to believe are valid. - - If, pursuant to or in connection with a single transaction or -arrangement, you convey, or propagate by procuring conveyance of, a -covered work, and grant a patent license to some of the parties -receiving the covered work authorizing them to use, propagate, modify -or convey a specific copy of the covered work, then the patent license -you grant is automatically extended to all recipients of the covered -work and works based on it. - - A patent license is "discriminatory" if it does not include within -the scope of its coverage, prohibits the exercise of, or is -conditioned on the non-exercise of one or more of the rights that are -specifically granted under this License. You may not convey a covered -work if you are a party to an arrangement with a third party that is -in the business of distributing software, under which you make payment -to the third party based on the extent of your activity of conveying -the work, and under which the third party grants, to any of the -parties who would receive the covered work from you, a discriminatory -patent license (a) in connection with copies of the covered work -conveyed by you (or copies made from those copies), or (b) primarily -for and in connection with specific products or compilations that -contain the covered work, unless you entered into that arrangement, -or that patent license was granted, prior to 28 March 2007. - - Nothing in this License shall be construed as excluding or limiting -any implied license or other defenses to infringement that may -otherwise be available to you under applicable patent law. - - 12. No Surrender of Others' Freedom. - - If conditions are imposed on you (whether by court order, agreement or -otherwise) that contradict the conditions of this License, they do not -excuse you from the conditions of this License. If you cannot convey a -covered work so as to satisfy simultaneously your obligations under this -License and any other pertinent obligations, then as a consequence you may -not convey it at all. For example, if you agree to terms that obligate you -to collect a royalty for further conveying from those to whom you convey -the Program, the only way you could satisfy both those terms and this -License would be to refrain entirely from conveying the Program. - - 13. Use with the GNU Affero General Public License. - - Notwithstanding any other provision of this License, you have -permission to link or combine any covered work with a work licensed -under version 3 of the GNU Affero General Public License into a single -combined work, and to convey the resulting work. The terms of this -License will continue to apply to the part which is the covered work, -but the special requirements of the GNU Affero General Public License, -section 13, concerning interaction through a network will apply to the -combination as such. - - 14. Revised Versions of this License. - - The Free Software Foundation may publish revised and/or new versions of -the GNU General Public License from time to time. Such new versions will -be similar in spirit to the present version, but may differ in detail to -address new problems or concerns. - - Each version is given a distinguishing version number. If the -Program specifies that a certain numbered version of the GNU General -Public License "or any later version" applies to it, you have the -option of following the terms and conditions either of that numbered -version or of any later version published by the Free Software -Foundation. If the Program does not specify a version number of the -GNU General Public License, you may choose any version ever published -by the Free Software Foundation. - - If the Program specifies that a proxy can decide which future -versions of the GNU General Public License can be used, that proxy's -public statement of acceptance of a version permanently authorizes you -to choose that version for the Program. - - Later license versions may give you additional or different -permissions. However, no additional obligations are imposed on any -author or copyright holder as a result of your choosing to follow a -later version. - - 15. Disclaimer of Warranty. - - THERE IS NO WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY -APPLICABLE LAW. EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT -HOLDERS AND/OR OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY -OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, -THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR -PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM -IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF -ALL NECESSARY SERVICING, REPAIR OR CORRECTION. - - 16. Limitation of Liability. - - IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING -WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MODIFIES AND/OR CONVEYS -THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY -GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE -USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF -DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD -PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), -EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF -SUCH DAMAGES. - - 17. Interpretation of Sections 15 and 16. - - If the disclaimer of warranty and limitation of liability provided -above cannot be given local legal effect according to their terms, -reviewing courts shall apply local law that most closely approximates -an absolute waiver of all civil liability in connection with the -Program, unless a warranty or assumption of liability accompanies a -copy of the Program in return for a fee. - - END OF TERMS AND CONDITIONS - - How to Apply These Terms to Your New Programs - - If you develop a new program, and you want it to be of the greatest -possible use to the public, the best way to achieve this is to make it -free software which everyone can redistribute and change under these terms. - - To do so, attach the following notices to the program. It is safest -to attach them to the start of each source file to most effectively -state the exclusion of warranty; and each file should have at least -the "copyright" line and a pointer to where the full notice is found. - - {one line to give the program's name and a brief idea of what it does.} - Copyright (C) {year} {name of author} - - This program is free software: you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation, either version 3 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. - - You should have received a copy of the GNU General Public License - along with this program. If not, see . - -Also add information on how to contact you by electronic and paper mail. - - If the program does terminal interaction, make it output a short -notice like this when it starts in an interactive mode: - - {project} Copyright (C) {year} {fullname} - This program comes with ABSOLUTELY NO WARRANTY; for details type `show w'. - This is free software, and you are welcome to redistribute it - under certain conditions; type `show c' for details. - -The hypothetical commands `show w' and `show c' should show the appropriate -parts of the General Public License. Of course, your program's commands -might be different; for a GUI interface, you would use an "about box". - - You should also get your employer (if you work as a programmer) or school, -if any, to sign a "copyright disclaimer" for the program, if necessary. -For more information on this, and how to apply and follow the GNU GPL, see -. - - The GNU General Public License does not permit incorporating your program -into proprietary programs. If your program is a subroutine library, you -may consider it more useful to permit linking proprietary applications with -the library. If this is what you want to do, use the GNU Lesser General -Public License instead of this License. But first, please read -. From 5a1824b4e39d6802551c8e1a6e7956d972af62a6 Mon Sep 17 00:00:00 2001 From: Don Inghram Date: Thu, 15 Jun 2023 07:17:52 -0600 Subject: [PATCH 055/348] [Issue 468] Add Support for NonDurable subscriptions (#992) --- perf/perf-consumer.go | 8 ++++ perf/pulsar-perf-go.go | 6 +++ pulsar/consumer.go | 4 ++ pulsar/consumer_impl.go | 2 +- pulsar/consumer_partition.go | 12 +++--- pulsar/consumer_test.go | 72 ++++++++++++++++++++++++++++++++++++ pulsar/reader_impl.go | 2 +- 7 files changed, 98 insertions(+), 8 deletions(-) diff --git a/perf/perf-consumer.go b/perf/perf-consumer.go index 825de62ff0..2172af533e 100644 --- a/perf/perf-consumer.go +++ b/perf/perf-consumer.go @@ -36,6 +36,8 @@ type ConsumeArgs struct { ReceiverQueueSize int EnableBatchIndexAck bool EnableAutoScaledReceiverQueueSize bool + SubscriptionMode pulsar.SubscriptionMode + SubscriptionType pulsar.SubscriptionType } func newConsumerCommand() *cobra.Command { @@ -60,6 +62,10 @@ func newConsumerCommand() *cobra.Command { flags.BoolVar(&consumeArgs.EnableBatchIndexAck, "enable-batch-index-ack", false, "Whether to enable batch index ACK") flags.BoolVar(&consumeArgs.EnableAutoScaledReceiverQueueSize, "enable-auto-scaled-queue-size", false, "Whether to enable auto scaled receiver queue size") + flags.IntVarP((*int)(&consumeArgs.SubscriptionMode), "subscription-mode", "m", int(pulsar.Durable), + "Subscription mode") + flags.IntVarP((*int)(&consumeArgs.SubscriptionType), "subscription-type", "t", int(pulsar.Exclusive), + "Subscription type") return cmd } @@ -83,6 +89,8 @@ func consume(consumeArgs *ConsumeArgs, stop <-chan struct{}) { SubscriptionName: consumeArgs.SubscriptionName, EnableBatchIndexAcknowledgment: consumeArgs.EnableBatchIndexAck, EnableAutoScaledReceiverQueueSize: consumeArgs.EnableAutoScaledReceiverQueueSize, + Type: consumeArgs.SubscriptionType, + SubscriptionMode: consumeArgs.SubscriptionMode, }) if err != nil { diff --git a/perf/pulsar-perf-go.go b/perf/pulsar-perf-go.go index a672a30271..40257e83c3 100644 --- a/perf/pulsar-perf-go.go +++ b/perf/pulsar-perf-go.go @@ -43,6 +43,8 @@ type ClientArgs struct { ServiceURL string TokenFile string TLSTrustCertFile string + TLSServerCertFile string + TLSServerKeyFile string MaxConnectionsPerBroker int } @@ -62,6 +64,8 @@ func NewClient() (pulsar.Client, error) { os.Exit(1) } clientOpts.Authentication = pulsar.NewAuthenticationToken(string(tokenBytes)) + } else if clientArgs.TLSServerCertFile != "" && clientArgs.TLSServerKeyFile != "" { + clientOpts.Authentication = pulsar.NewAuthenticationTLS(clientArgs.TLSServerCertFile, clientArgs.TLSServerKeyFile) } if clientArgs.TLSTrustCertFile != "" { @@ -97,6 +101,8 @@ func main() { flags.StringVarP(&clientArgs.ServiceURL, "service-url", "u", "pulsar://localhost:6650", "The Pulsar service URL") flags.StringVar(&clientArgs.TokenFile, "token-file", "", "file path to the Pulsar JWT file") + flags.StringVar(&clientArgs.TLSServerCertFile, "cert-file", "", "file path to the TLS authentication cert") + flags.StringVar(&clientArgs.TLSServerKeyFile, "key-file", "", "file path to the TLS authentication key") flags.StringVar(&clientArgs.TLSTrustCertFile, "trust-cert-file", "", "file path to the trusted certificate file") flags.IntVarP(&clientArgs.MaxConnectionsPerBroker, "max-connections", "c", 1, "Max connections to open to broker. Defaults to 1.") diff --git a/pulsar/consumer.go b/pulsar/consumer.go index 3ef72c7ca8..a62eabe1ed 100644 --- a/pulsar/consumer.go +++ b/pulsar/consumer.go @@ -242,6 +242,10 @@ type ConsumerOptions struct { // NOTE: This option does not work if AckWithResponse is true // because there are only synchronous APIs for acknowledgment AckGroupingOptions *AckGroupingOptions + + // SubscriptionMode specifies the subscription mode to be used when subscribing to a topic. + // Default is `Durable` + SubscriptionMode SubscriptionMode } // Consumer is an interface that abstracts behavior of Pulsar's consumer diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index 07f38c3633..d782beab5e 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -385,7 +385,7 @@ func (c *consumer) internalTopicSubscribeToPartitions() error { subProperties: subProperties, replicateSubscriptionState: c.options.ReplicateSubscriptionState, startMessageID: nil, - subscriptionMode: durable, + subscriptionMode: c.options.SubscriptionMode, readCompacted: c.options.ReadCompacted, interceptors: c.options.Interceptors, maxReconnectToBroker: c.options.MaxReconnectToBroker, diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 6e241d8f7f..2d2a1940f7 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -65,15 +65,15 @@ func (s consumerState) String() string { } } -type subscriptionMode int +type SubscriptionMode int const ( // Make the subscription to be backed by a durable cursor that will retain messages and persist the current // position - durable subscriptionMode = iota + Durable SubscriptionMode = iota // Lightweight subscription mode that doesn't have a durable cursor associated - nonDurable + NonDurable ) const ( @@ -101,7 +101,7 @@ type partitionConsumerOpts struct { replicateSubscriptionState bool startMessageID *trackingMessageID startMessageIDInclusive bool - subscriptionMode subscriptionMode + subscriptionMode SubscriptionMode readCompacted bool disableForceTopicCreation bool interceptors ConsumerInterceptors @@ -1698,7 +1698,7 @@ func (pc *partitionConsumer) grabConn() error { RequestId: proto.Uint64(requestID), ConsumerName: proto.String(pc.name), PriorityLevel: nil, - Durable: proto.Bool(pc.options.subscriptionMode == durable), + Durable: proto.Bool(pc.options.subscriptionMode == Durable), Metadata: internal.ConvertFromStringMap(pc.options.metadata), SubscriptionProperties: internal.ConvertFromStringMap(pc.options.subProperties), ReadCompacted: proto.Bool(pc.options.readCompacted), @@ -1709,7 +1709,7 @@ func (pc *partitionConsumer) grabConn() error { } pc.startMessageID.set(pc.clearReceiverQueue()) - if pc.options.subscriptionMode != durable { + if pc.options.subscriptionMode != Durable { // For regular subscriptions the broker will determine the restarting point cmdSubscribe.StartMessageId = convertToMessageIDData(pc.startMessageID.get()) } diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 521e576767..2d785b4b23 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -4074,6 +4074,78 @@ func TestConsumerWithAutoScaledQueueReceive(t *testing.T) { }) } +func TestConsumerNonDurable(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + + topicName := newTopicName() + ctx := context.Background() + + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topicName, + }) + assert.Nil(t, err) + defer producer.Close() + + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topicName, + SubscriptionName: "sub-1", + Type: Shared, + SubscriptionMode: NonDurable, + }) + assert.Nil(t, err) + + i := 1 + if _, err := producer.Send(ctx, &ProducerMessage{ + Payload: []byte(fmt.Sprintf("msg-content-%d", i)), + }); err != nil { + t.Fatal(err) + } + + msg, err := consumer.Receive(ctx) + assert.Nil(t, err) + assert.Equal(t, fmt.Sprintf("msg-content-%d", i), string(msg.Payload())) + consumer.Ack(msg) + + consumer.Close() + + i++ + + // send a message. Pulsar should delete it as there is no active subscription + if _, err := producer.Send(ctx, &ProducerMessage{ + Payload: []byte(fmt.Sprintf("msg-content-%d", i)), + }); err != nil { + t.Fatal(err) + } + + i++ + + // Subscribe again + consumer, err = client.Subscribe(ConsumerOptions{ + Topic: topicName, + SubscriptionName: "sub-1", + Type: Shared, + SubscriptionMode: NonDurable, + }) + assert.Nil(t, err) + defer consumer.Close() + + if _, err := producer.Send(ctx, &ProducerMessage{ + Payload: []byte(fmt.Sprintf("msg-content-%d", i)), + }); err != nil { + t.Fatal(err) + } + + msg, err = consumer.Receive(ctx) + assert.Nil(t, err) + assert.Equal(t, fmt.Sprintf("msg-content-%d", i), string(msg.Payload())) + consumer.Ack(msg) +} + func TestConsumerBatchIndexAckDisabled(t *testing.T) { client, err := NewClient(ClientOptions{ URL: lookupURL, diff --git a/pulsar/reader_impl.go b/pulsar/reader_impl.go index 36b492abea..5a2128a377 100644 --- a/pulsar/reader_impl.go +++ b/pulsar/reader_impl.go @@ -106,7 +106,7 @@ func newReader(client *client, options ReaderOptions) (Reader, error) { receiverQueueSize: receiverQueueSize, startMessageID: startMessageID, startMessageIDInclusive: options.StartMessageIDInclusive, - subscriptionMode: nonDurable, + subscriptionMode: NonDurable, readCompacted: options.ReadCompacted, metadata: options.Properties, nackRedeliveryDelay: defaultNackRedeliveryDelay, From 6acecf06aee5ab987b5e527d24a2aee7f82f3598 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 16 Jun 2023 13:24:56 +0800 Subject: [PATCH 056/348] Fix broken master by upgrading JRE to 17 (#1030) --- Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Dockerfile b/Dockerfile index f66eba54b5..7548729c76 100644 --- a/Dockerfile +++ b/Dockerfile @@ -25,7 +25,7 @@ ARG GOLANG_IMAGE=golang:latest FROM $PULSAR_IMAGE as pulsar FROM $GOLANG_IMAGE -RUN apt-get update && apt-get install -y openjdk-11-jre-headless ca-certificates +RUN apt-get update && apt-get install -y openjdk-17-jre-headless ca-certificates COPY --from=pulsar /pulsar /pulsar From 7f91b2bcd798e1dc6ff27b10a557c8ee5440a83a Mon Sep 17 00:00:00 2001 From: grayson <916028390@qq.com> Date: Fri, 16 Jun 2023 17:43:33 +0800 Subject: [PATCH 057/348] [Issue 1027][producer] fix: split sendRequest and make reconnectToBroker and other operate in the same coroutine (#1029) --- pulsar/producer_partition.go | 38 ++++++++++++------------------------ 1 file changed, 13 insertions(+), 25 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index c4a460eed4..6bd90818e2 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -84,8 +84,8 @@ type partitionProducer struct { compressionProvider compression.Provider // Channel where app is posting messages to be published - eventsChan chan interface{} - closeCh chan struct{} + dataChan chan *sendRequest + cmdChan chan interface{} connectClosedCh chan connectionClosed publishSemaphore internal.Semaphore @@ -150,9 +150,9 @@ func newPartitionProducer(client *client, topic string, options *ProducerOptions log: logger, options: options, producerID: client.rpcClient.NewProducerID(), - eventsChan: make(chan interface{}, maxPendingMessages), + dataChan: make(chan *sendRequest, maxPendingMessages), + cmdChan: make(chan interface{}, 10), connectClosedCh: make(chan connectionClosed, 10), - closeCh: make(chan struct{}), batchFlushTicker: time.NewTicker(batchingMaxPublishDelay), compressionProvider: internal.GetCompressionProvider(pb.CompressionType(options.CompressionType), compression.Level(options.CompressionLevel)), @@ -438,31 +438,21 @@ func (p *partitionProducer) reconnectToBroker() { } func (p *partitionProducer) runEventsLoop() { - go func() { - for { - select { - case <-p.closeCh: - p.log.Info("close producer, exit reconnect") - return - case <-p.connectClosedCh: - p.log.Info("runEventsLoop will reconnect in producer") - p.reconnectToBroker() - } - } - }() - for { select { - case i := <-p.eventsChan: + case data := <-p.dataChan: + p.internalSend(data) + case i := <-p.cmdChan: switch v := i.(type) { - case *sendRequest: - p.internalSend(v) case *flushRequest: p.internalFlush(v) case *closeProducer: p.internalClose(v) return } + case <-p.connectClosedCh: + p.log.Info("runEventsLoop will reconnect in producer") + p.reconnectToBroker() case <-p.batchFlushTicker.C: p.internalFlushCurrentBatch() } @@ -1165,7 +1155,7 @@ func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *Producer } p.options.Interceptors.BeforeSend(p, msg) - p.eventsChan <- sr + p.dataChan <- sr if !p.options.DisableBlockIfQueueFull { // block if queue full @@ -1304,8 +1294,6 @@ func (p *partitionProducer) internalClose(req *closeProducer) { p.setProducerState(producerClosed) p._getConn().UnregisterListener(p.producerID) p.batchFlushTicker.Stop() - - close(p.closeCh) } func (p *partitionProducer) LastSequenceID() int64 { @@ -1317,7 +1305,7 @@ func (p *partitionProducer) Flush() error { doneCh: make(chan struct{}), err: nil, } - p.eventsChan <- flushReq + p.cmdChan <- flushReq // wait for the flush request to complete <-flushReq.doneCh @@ -1345,7 +1333,7 @@ func (p *partitionProducer) Close() { } cp := &closeProducer{doneCh: make(chan struct{})} - p.eventsChan <- cp + p.cmdChan <- cp // wait for close producer request to complete <-cp.doneCh From 56c96911789faba94b0ca7a7f62ce2504271bd6a Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Mon, 26 Jun 2023 12:08:02 +0800 Subject: [PATCH 058/348] fix: install openjdk-17 in Dockerfile (#1037) --- Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Dockerfile b/Dockerfile index 7548729c76..07eff8ea2d 100644 --- a/Dockerfile +++ b/Dockerfile @@ -25,7 +25,7 @@ ARG GOLANG_IMAGE=golang:latest FROM $PULSAR_IMAGE as pulsar FROM $GOLANG_IMAGE -RUN apt-get update && apt-get install -y openjdk-17-jre-headless ca-certificates +RUN apt-get update && apt-get install -y openjdk-17-jre ca-certificates COPY --from=pulsar /pulsar /pulsar From 5f825166530bdb89ae2cb785511092b98038d78e Mon Sep 17 00:00:00 2001 From: Don Inghram Date: Tue, 27 Jun 2023 14:40:21 -0600 Subject: [PATCH 059/348] Allow user to specify TLS ciphers an min/max TLS version (#1041) --- pulsar/client.go | 9 +++++++++ pulsar/client_impl.go | 3 +++ pulsar/internal/connection.go | 6 ++++++ pulsar/internal/http_client.go | 3 +++ 4 files changed, 21 insertions(+) diff --git a/pulsar/client.go b/pulsar/client.go index 7e6725d42e..d6f18c9aa9 100644 --- a/pulsar/client.go +++ b/pulsar/client.go @@ -120,6 +120,15 @@ type ClientOptions struct { // Configure whether the Pulsar client verify the validity of the host name from broker (default: false) TLSValidateHostname bool + // TLSCipherSuites is a list of enabled TLS 1.0–1.2 cipher suites. See tls.Config CipherSuites for more information. + TLSCipherSuites []uint16 + + // TLSMinVersion contains the minimum TLS version that is acceptable. See tls.Config MinVersion for more information. + TLSMinVersion uint16 + + // TLSMaxVersion contains the maximum TLS version that is acceptable. See tls.Config MaxVersion for more information. + TLSMaxVersion uint16 + // Configure the net model for vpc user to connect the pulsar broker ListenerName string diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index 9b44987949..c283f5a827 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -91,6 +91,9 @@ func newClient(options ClientOptions) (Client, error) { TrustCertsFilePath: options.TLSTrustCertsFilePath, ValidateHostname: options.TLSValidateHostname, ServerName: url.Hostname(), + CipherSuites: options.TLSCipherSuites, + MinVersion: options.TLSMinVersion, + MaxVersion: options.TLSMaxVersion, } default: return nil, newError(InvalidConfiguration, fmt.Sprintf("Invalid URL scheme '%s'", url.Scheme)) diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 59aad1675e..1f80d20d33 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -50,6 +50,9 @@ type TLSOptions struct { AllowInsecureConnection bool ValidateHostname bool ServerName string + CipherSuites []uint16 + MinVersion uint16 + MaxVersion uint16 } var ( @@ -1046,6 +1049,9 @@ func (c *connection) closed() bool { func (c *connection) getTLSConfig() (*tls.Config, error) { tlsConfig := &tls.Config{ InsecureSkipVerify: c.tlsOptions.AllowInsecureConnection, + CipherSuites: c.tlsOptions.CipherSuites, + MinVersion: c.tlsOptions.MinVersion, + MaxVersion: c.tlsOptions.MaxVersion, } if c.tlsOptions.TrustCertsFilePath != "" { diff --git a/pulsar/internal/http_client.go b/pulsar/internal/http_client.go index 2d440d6f9a..e68bd17c39 100644 --- a/pulsar/internal/http_client.go +++ b/pulsar/internal/http_client.go @@ -339,6 +339,9 @@ func getDefaultTransport(tlsConfig *TLSOptions) (http.RoundTripper, error) { if tlsConfig != nil { cfg := &tls.Config{ InsecureSkipVerify: tlsConfig.AllowInsecureConnection, + CipherSuites: tlsConfig.CipherSuites, + MinVersion: tlsConfig.MinVersion, + MaxVersion: tlsConfig.MaxVersion, } if len(tlsConfig.TrustCertsFilePath) > 0 { rootCA, err := os.ReadFile(tlsConfig.TrustCertsFilePath) From 3a4e5cfbf2c4abe3bf5df137a970ffd70f8efded Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Wed, 28 Jun 2023 10:58:26 +0800 Subject: [PATCH 060/348] Add single partition router (#999) * add single-partition-router * fix license * fix ci * Modify as sync.Once * Update pulsar/producer_test.go Co-authored-by: Zixuan Liu * Modify with CR * Verify message was published on single partition --------- Co-authored-by: Zixuan Liu --- pulsar/producer_test.go | 61 +++++++++++++++++++++++++ pulsar/single_partition_router.go | 42 +++++++++++++++++ pulsar/single_partition_router_test.go | 62 ++++++++++++++++++++++++++ 3 files changed, 165 insertions(+) create mode 100644 pulsar/single_partition_router.go create mode 100644 pulsar/single_partition_router_test.go diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index b587975c9c..2721fa3d80 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -560,6 +560,67 @@ func TestMessageRouter(t *testing.T) { assert.NotNil(t, msg) assert.Equal(t, string(msg.Payload()), "hello") } +func TestMessageSingleRouter(t *testing.T) { + // Create topic with 5 partitions + topicAdminURL := "admin/v2/persistent/public/default/my-single-partitioned-topic/partitions" + err := httpPut(topicAdminURL, 5) + defer httpDelete(topicAdminURL) + if err != nil { + t.Fatal(err) + } + client, err := NewClient(ClientOptions{ + URL: serviceURL, + }) + + assert.Nil(t, err) + defer client.Close() + + numOfMessages := 10 + + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: "my-single-partitioned-topic", + SubscriptionName: "my-sub", + }) + + assert.Nil(t, err) + defer consumer.Close() + + producer, err := client.CreateProducer(ProducerOptions{ + Topic: "my-single-partitioned-topic", + MessageRouter: NewSinglePartitionRouter(), + }) + + assert.Nil(t, err) + defer producer.Close() + + ctx := context.Background() + + for i := 0; i < numOfMessages; i++ { + ID, err := producer.Send(ctx, &ProducerMessage{ + Payload: []byte("hello"), + }) + assert.Nil(t, err) + assert.NotNil(t, ID) + } + + // Verify message was published on single partition + msgCount := 0 + msgPartitionMap := make(map[string]int) + for i := 0; i < numOfMessages; i++ { + msg, err := consumer.Receive(ctx) + assert.Nil(t, err) + assert.NotNil(t, msg) + consumer.Ack(msg) + msgCount++ + msgPartitionMap[msg.Topic()]++ + } + assert.Equal(t, msgCount, numOfMessages) + assert.Equal(t, len(msgPartitionMap), 1) + for _, i := range msgPartitionMap { + assert.Equal(t, i, numOfMessages) + } + +} func TestNonPersistentTopic(t *testing.T) { topicName := "non-persistent://public/default/testNonPersistentTopic" diff --git a/pulsar/single_partition_router.go b/pulsar/single_partition_router.go new file mode 100644 index 0000000000..7896aa3246 --- /dev/null +++ b/pulsar/single_partition_router.go @@ -0,0 +1,42 @@ +// 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 pulsar + +import "sync" + +func NewSinglePartitionRouter() func(*ProducerMessage, TopicMetadata) int { + var ( + singlePartition *int + once sync.Once + ) + return func(message *ProducerMessage, metadata TopicMetadata) int { + numPartitions := metadata.NumPartitions() + if len(message.Key) != 0 { + // When a key is specified, use the hash of that key + return int(getHashingFunction(JavaStringHash)(message.Key) % numPartitions) + } + once.Do(func() { + partition := r.R.Intn(int(numPartitions)) + singlePartition = &partition + }) + + return *singlePartition + + } + +} diff --git a/pulsar/single_partition_router_test.go b/pulsar/single_partition_router_test.go new file mode 100644 index 0000000000..a5a42156fc --- /dev/null +++ b/pulsar/single_partition_router_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 pulsar + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +type topicMetaData struct { + partition uint32 +} + +func (t topicMetaData) NumPartitions() uint32 { + return t.partition +} + +func TestNewSinglePartitionRouter(t *testing.T) { + numPartitions := topicMetaData{2} + router := NewSinglePartitionRouter() + p := router(&ProducerMessage{ + Payload: []byte("message 2"), + }, numPartitions) + assert.GreaterOrEqual(t, p, 0) + + p2 := router(&ProducerMessage{ + Payload: []byte("message 2"), + }, numPartitions) + assert.Equal(t, p, p2) +} + +func TestNewSinglePartitionRouterWithKey(t *testing.T) { + router := NewSinglePartitionRouter() + numPartitions := topicMetaData{3} + p := router(&ProducerMessage{ + Payload: []byte("message 2"), + Key: "my-key", + }, numPartitions) + assert.Equal(t, 1, p) + + p2 := router(&ProducerMessage{ + Key: "my-key", + Payload: []byte("message 2"), + }, numPartitions) + assert.Equal(t, p, p2) +} From 3da3e5d43134e7ccca933c6a80d6a20fb926e05a Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 28 Jun 2023 11:06:57 +0800 Subject: [PATCH 061/348] [fix] Fix ordering key not being set and parsed when batching is disabled (#1034) --- pulsar/consumer_partition.go | 1 + pulsar/producer_partition.go | 4 ++++ pulsar/producer_test.go | 22 ++++++++++++++++++---- 3 files changed, 23 insertions(+), 4 deletions(-) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 2d2a1940f7..cd7aa50199 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -1190,6 +1190,7 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header redeliveryCount: response.GetRedeliveryCount(), schemaVersion: msgMeta.GetSchemaVersion(), schemaInfoCache: pc.schemaInfoCache, + orderingKey: string(msgMeta.GetOrderingKey()), index: messageIndex, brokerPublishTime: brokerPublishTime, } diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 6bd90818e2..837d1d78e6 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -733,6 +733,10 @@ func (p *partitionProducer) genMetadata(msg *ProducerMessage, mm.PartitionKey = proto.String(msg.Key) } + if len(msg.OrderingKey) != 0 { + mm.OrderingKey = []byte(msg.OrderingKey) + } + if msg.Properties != nil { mm.Properties = internal.ConvertFromStringMap(msg.Properties) } diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 2721fa3d80..7c3abdc721 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -1967,7 +1967,15 @@ func TestMemLimitContextCancel(t *testing.T) { assert.NoError(t, err) } -func TestSendMessagesWithMetadata(t *testing.T) { +func TestBatchSendMessagesWithMetadata(t *testing.T) { + testSendMessagesWithMetadata(t, false) +} + +func TestNoBatchSendMessagesWithMetadata(t *testing.T) { + testSendMessagesWithMetadata(t, true) +} + +func testSendMessagesWithMetadata(t *testing.T, disableBatch bool) { client, err := NewClient(ClientOptions{ URL: lookupURL, }) @@ -1978,7 +1986,7 @@ func TestSendMessagesWithMetadata(t *testing.T) { topic := newTopicName() producer, err := client.CreateProducer(ProducerOptions{ Topic: topic, - DisableBatching: true, + DisableBatching: disableBatch, }) assert.Nil(t, err) @@ -1989,7 +1997,10 @@ func TestSendMessagesWithMetadata(t *testing.T) { assert.Nil(t, err) msg := &ProducerMessage{EventTime: time.Now().Local(), - Payload: []byte("msg")} + Key: "my-key", + OrderingKey: "my-ordering-key", + Properties: map[string]string{"k1": "v1", "k2": "v2"}, + Payload: []byte("msg")} _, err = producer.Send(context.Background(), msg) assert.Nil(t, err) @@ -1997,5 +2008,8 @@ func TestSendMessagesWithMetadata(t *testing.T) { recvMsg, err := consumer.Receive(context.Background()) assert.Nil(t, err) - assert.Equal(t, internal.TimestampMillis(recvMsg.EventTime()), internal.TimestampMillis(msg.EventTime)) + assert.Equal(t, internal.TimestampMillis(msg.EventTime), internal.TimestampMillis(recvMsg.EventTime())) + assert.Equal(t, msg.Key, recvMsg.Key()) + assert.Equal(t, msg.OrderingKey, recvMsg.OrderingKey()) + assert.Equal(t, msg.Properties, recvMsg.Properties()) } From 6f01a7cead8704aa59afcd819545512d0259af07 Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Mon, 3 Jul 2023 09:44:33 +0800 Subject: [PATCH 062/348] [Fix] check if callback is nil before calling it (#1036) Co-authored-by: gunli --- pulsar/producer_partition.go | 58 +++++++++++++++++++----------------- 1 file changed, 30 insertions(+), 28 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 837d1d78e6..98c6c980dd 100644 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -467,6 +467,14 @@ func (p *partitionProducer) Name() string { return p.producerName } +func runCallback(cb func(MessageID, *ProducerMessage, error), id MessageID, msg *ProducerMessage, err error) { + if cb == nil { + return + } + + cb(id, msg, err) +} + func (p *partitionProducer) internalSend(request *sendRequest) { p.log.Debug("Received send request: ", *request.msg) @@ -480,7 +488,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { var err error if msg.Value != nil && msg.Payload != nil { p.log.Error("Can not set Value and Payload both") - request.callback(nil, request.msg, errors.New("can not set Value and Payload both")) + runCallback(request.callback, nil, request.msg, errors.New("can not set Value and Payload both")) return } @@ -494,7 +502,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { if msg.Schema != nil && p.options.Schema != nil && msg.Schema.GetSchemaInfo().hash() != p.options.Schema.GetSchemaInfo().hash() { p.releaseSemaphoreAndMem(uncompressedPayloadSize) - request.callback(nil, request.msg, fmt.Errorf("msg schema can not match with producer schema")) + runCallback(request.callback, nil, request.msg, fmt.Errorf("msg schema can not match with producer schema")) p.log.WithError(err).Errorf("The producer %s of the topic %s is disabled the `MultiSchema`", p.producerName, p.topic) return } @@ -513,7 +521,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { schemaPayload, err = schema.Encode(msg.Value) if err != nil { p.releaseSemaphoreAndMem(uncompressedPayloadSize) - request.callback(nil, request.msg, newError(SchemaFailure, err.Error())) + runCallback(request.callback, nil, request.msg, newError(SchemaFailure, err.Error())) p.log.WithError(err).Errorf("Schema encode message failed %s", msg.Value) return } @@ -530,7 +538,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { if err != nil { p.releaseSemaphoreAndMem(uncompressedPayloadSize) p.log.WithError(err).Error("get schema version fail") - request.callback(nil, request.msg, fmt.Errorf("get schema version fail, err: %w", err)) + runCallback(request.callback, nil, request.msg, fmt.Errorf("get schema version fail, err: %w", err)) return } p.schemaCache.Put(schema.GetSchemaInfo(), schemaVersion) @@ -589,7 +597,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { // if msg is too large and chunking is disabled if checkSize > maxMessageSize && !p.options.EnableChunking { p.releaseSemaphoreAndMem(uncompressedPayloadSize) - request.callback(nil, request.msg, errMessageTooLarge) + runCallback(request.callback, nil, request.msg, errMessageTooLarge) p.log.WithError(errMessageTooLarge). WithField("size", checkSize). WithField("properties", msg.Properties). @@ -608,7 +616,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { payloadChunkSize = int(p._getConn().GetMaxMessageSize()) - proto.Size(mm) if payloadChunkSize <= 0 { p.releaseSemaphoreAndMem(uncompressedPayloadSize) - request.callback(nil, msg, errMetaTooLarge) + runCallback(request.callback, nil, msg, errMetaTooLarge) p.log.WithError(errMetaTooLarge). WithField("metadata size", proto.Size(mm)). WithField("properties", msg.Properties). @@ -683,7 +691,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { if ok := addRequestToBatch(smm, p, uncompressedPayload, request, msg, deliverAt, schemaVersion, multiSchemaEnabled); !ok { p.releaseSemaphoreAndMem(uncompressedPayloadSize) - request.callback(nil, request.msg, errFailAddToBatch) + runCallback(request.callback, nil, request.msg, errFailAddToBatch) p.log.WithField("size", uncompressedSize). WithField("properties", msg.Properties). Error("unable to add message to batch") @@ -835,7 +843,7 @@ func (p *partitionProducer) internalSingleSend(mm *pb.MessageMetadata, ) } if err != nil { - request.callback(nil, request.msg, err) + runCallback(request.callback, nil, request.msg, err) p.releaseSemaphoreAndMem(int64(len(msg.Payload))) p.log.WithError(err).Errorf("Single message serialize failed %s", msg.Value) return @@ -875,7 +883,7 @@ func (p *partitionProducer) internalFlushCurrentBatch() { if err != nil { for _, cb := range callbacks { if sr, ok := cb.(*sendRequest); ok { - sr.callback(nil, sr.msg, err) + runCallback(sr.callback, nil, sr.msg, err) } } if errors.Is(err, internal.ErrExceedMaxMessageSize) { @@ -985,7 +993,7 @@ func (p *partitionProducer) failTimeoutMessages() { if sr.callback != nil { sr.callbackOnce.Do(func() { - sr.callback(nil, sr.msg, errSendTimeout) + runCallback(sr.callback, nil, sr.msg, errSendTimeout) }) } if sr.transaction != nil { @@ -1018,7 +1026,7 @@ func (p *partitionProducer) internalFlushCurrentBatches() { if errs[i] != nil { for _, cb := range callbacks[i] { if sr, ok := cb.(*sendRequest); ok { - sr.callback(nil, sr.msg, errs[i]) + runCallback(sr.callback, nil, sr.msg, errs[i]) } } if errors.Is(errs[i], internal.ErrExceedMaxMessageSize) { @@ -1106,26 +1114,26 @@ func (p *partitionProducer) SendAsync(ctx context.Context, msg *ProducerMessage, func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *ProducerMessage, callback func(MessageID, *ProducerMessage, error), flushImmediately bool) { - //Register transaction operation to transaction and the transaction coordinator. + // Register transaction operation to transaction and the transaction coordinator. var newCallback func(MessageID, *ProducerMessage, error) if msg.Transaction != nil { transactionImpl := (msg.Transaction).(*transaction) if transactionImpl.state != TxnOpen { p.log.WithField("state", transactionImpl.state).Error("Failed to send message" + " by a non-open transaction.") - callback(nil, msg, newError(InvalidStatus, "Failed to send message by a non-open transaction.")) + runCallback(callback, nil, msg, newError(InvalidStatus, "Failed to send message by a non-open transaction.")) return } if err := transactionImpl.registerProducerTopic(p.topic); err != nil { - callback(nil, msg, err) + runCallback(callback, nil, msg, err) return } if err := transactionImpl.registerSendOrAckOp(); err != nil { - callback(nil, msg, err) + runCallback(callback, nil, msg, err) } newCallback = func(id MessageID, producerMessage *ProducerMessage, err error) { - callback(id, producerMessage, err) + runCallback(callback, id, producerMessage, err) transactionImpl.endSendOrAckOp(err) } } else { @@ -1133,7 +1141,7 @@ func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *Producer } if p.getProducerState() != producerReady { // Producer is closing - newCallback(nil, msg, errProducerClosed) + runCallback(newCallback, nil, msg, errProducerClosed) return } @@ -1253,9 +1261,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) } if sr.totalChunks <= 1 || sr.chunkID == sr.totalChunks-1 { - if sr.callback != nil { - sr.callback(msgID, sr.msg, nil) - } + runCallback(sr.callback, msgID, sr.msg, nil) p.options.Interceptors.OnSendAcknowledgement(p, sr.msg, msgID) } } @@ -1406,27 +1412,23 @@ func (p *partitionProducer) releaseSemaphoreAndMem(size int64) { func (p *partitionProducer) canAddToQueue(sr *sendRequest, uncompressedPayloadSize int64) bool { if p.options.DisableBlockIfQueueFull { if !p.publishSemaphore.TryAcquire() { - if sr.callback != nil { - sr.callback(nil, sr.msg, errSendQueueIsFull) - } + runCallback(sr.callback, nil, sr.msg, errSendQueueIsFull) return false } if !p.client.memLimit.TryReserveMemory(uncompressedPayloadSize) { p.publishSemaphore.Release() - if sr.callback != nil { - sr.callback(nil, sr.msg, errMemoryBufferIsFull) - } + runCallback(sr.callback, nil, sr.msg, errMemoryBufferIsFull) return false } } else { if !p.publishSemaphore.Acquire(sr.ctx) { - sr.callback(nil, sr.msg, errContextExpired) + runCallback(sr.callback, nil, sr.msg, errContextExpired) return false } if !p.client.memLimit.ReserveMemory(sr.ctx, uncompressedPayloadSize) { p.publishSemaphore.Release() - sr.callback(nil, sr.msg, errContextExpired) + runCallback(sr.callback, nil, sr.msg, errContextExpired) return false } } From 5f8df2782251226b63b2dad9f4e87fd4bbd31ef5 Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Mon, 3 Jul 2023 09:57:14 +0800 Subject: [PATCH 063/348] [Refactor] refactor duplicated code lines and fix typo errors (#1039) * [Refactor] refactor duplicated code lines and fix typo errors * [typo] revert * [typo] revert * [typo] revert * [refactor] delete redunpdant code lines * [typo] revert some words * [fix] set useTxn when use transaction --------- Co-authored-by: gunli --- pulsar/producer_partition.go | 94 +++++++++++++++++------------------- 1 file changed, 45 insertions(+), 49 deletions(-) mode change 100644 => 100755 pulsar/producer_partition.go diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go old mode 100644 new mode 100755 index 98c6c980dd..9d04427a88 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -710,17 +710,19 @@ func addRequestToBatch(smm *pb.SingleMessageMetadata, p *partitionProducer, uncompressedPayload []byte, request *sendRequest, msg *ProducerMessage, deliverAt time.Time, schemaVersion []byte, multiSchemaEnabled bool) bool { - var ok bool + var useTxn bool + var mostSigBits uint64 + var leastSigBits uint64 if request.transaction != nil { txnID := request.transaction.GetTxnID() - ok = p.batchBuilder.Add(smm, p.sequenceIDGenerator, uncompressedPayload, request, - msg.ReplicationClusters, deliverAt, schemaVersion, multiSchemaEnabled, true, txnID.MostSigBits, - txnID.LeastSigBits) - } else { - ok = p.batchBuilder.Add(smm, p.sequenceIDGenerator, uncompressedPayload, request, - msg.ReplicationClusters, deliverAt, schemaVersion, multiSchemaEnabled, false, 0, 0) + useTxn = true + mostSigBits = txnID.MostSigBits + leastSigBits = txnID.LeastSigBits } - return ok + + return p.batchBuilder.Add(smm, p.sequenceIDGenerator, uncompressedPayload, request, + msg.ReplicationClusters, deliverAt, schemaVersion, multiSchemaEnabled, useTxn, mostSigBits, + leastSigBits) } func (p *partitionProducer) genMetadata(msg *ProducerMessage, @@ -764,6 +766,14 @@ func (p *partitionProducer) updateMetadataSeqID(mm *pb.MessageMetadata, msg *Pro } } +func (p *partitionProducer) updateSingleMessageMetadataSeqID(smm *pb.SingleMessageMetadata, msg *ProducerMessage) { + if msg.SequenceID != nil { + smm.SequenceId = proto.Uint64(uint64(*msg.SequenceID)) + } else { + smm.SequenceId = proto.Uint64(internal.GetAndAdd(p.sequenceIDGenerator, 1)) + } +} + func (p *partitionProducer) genSingleMessageMetadataInBatch(msg *ProducerMessage, uncompressedSize int) (smm *pb.SingleMessageMetadata) { smm = &pb.SingleMessageMetadata{ @@ -786,14 +796,7 @@ func (p *partitionProducer) genSingleMessageMetadataInBatch(msg *ProducerMessage smm.Properties = internal.ConvertFromStringMap(msg.Properties) } - var sequenceID uint64 - if msg.SequenceID != nil { - sequenceID = uint64(*msg.SequenceID) - } else { - sequenceID = internal.GetAndAdd(p.sequenceIDGenerator, 1) - } - - smm.SequenceId = proto.Uint64(sequenceID) + p.updateSingleMessageMetadataSeqID(smm, msg) return } @@ -813,35 +816,30 @@ func (p *partitionProducer) internalSingleSend(mm *pb.MessageMetadata, } sid := *mm.SequenceId - var err error + var useTxn bool + var mostSigBits uint64 + var leastSigBits uint64 + if request.transaction != nil { txnID := request.transaction.GetTxnID() - err = internal.SingleSend( - buffer, - p.producerID, - sid, - mm, - payloadBuf, - p.encryptor, - maxMessageSize, - true, - txnID.MostSigBits, - txnID.LeastSigBits, - ) - } else { - err = internal.SingleSend( - buffer, - p.producerID, - sid, - mm, - payloadBuf, - p.encryptor, - maxMessageSize, - false, - 0, - 0, - ) - } + useTxn = true + mostSigBits = txnID.MostSigBits + leastSigBits = txnID.LeastSigBits + } + + err := internal.SingleSend( + buffer, + p.producerID, + sid, + mm, + payloadBuf, + p.encryptor, + maxMessageSize, + useTxn, + mostSigBits, + leastSigBits, + ) + if err != nil { runCallback(request.callback, nil, request.msg, err) p.releaseSemaphoreAndMem(int64(len(msg.Payload))) @@ -1001,7 +999,7 @@ func (p *partitionProducer) failTimeoutMessages() { } } - // flag the send has completed with error, flush make no effect + // flag the sending has completed with error, flush make no effect pi.Complete() pi.Unlock() @@ -1116,8 +1114,10 @@ func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *Producer callback func(MessageID, *ProducerMessage, error), flushImmediately bool) { // Register transaction operation to transaction and the transaction coordinator. var newCallback func(MessageID, *ProducerMessage, error) + var txn *transaction if msg.Transaction != nil { transactionImpl := (msg.Transaction).(*transaction) + txn = transactionImpl if transactionImpl.state != TxnOpen { p.log.WithField("state", transactionImpl.state).Error("Failed to send message" + " by a non-open transaction.") @@ -1150,10 +1150,6 @@ func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *Producer // callbackOnce make sure the callback is only invoked once in chunking callbackOnce := &sync.Once{} - var txn *transaction - if msg.Transaction != nil { - txn = (msg.Transaction).(*transaction) - } sr := &sendRequest{ ctx: ctx, msg: msg, @@ -1398,7 +1394,7 @@ func (p *partitionProducer) _setConn(conn internal.Connection) { // _getConn returns internal connection field of this partition producer atomically. // Note: should only be called by this partition producer before attempting to use the connection func (p *partitionProducer) _getConn() internal.Connection { - // Invariant: The conn must be non-nill for the lifetime of the partitionProducer. + // Invariant: The conn must be non-nil for the lifetime of the partitionProducer. // For this reason we leave this cast unchecked and panic() if the // invariant is broken return p.conn.Load().(internal.Connection) From 163cd7e5a7666ed59796608ccf0cb20fed76511d Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Tue, 4 Jul 2023 16:42:11 +0800 Subject: [PATCH 064/348] [Improve] improve the perf of schema and schema cache (#1033) * [Improve] improve the perf of schema and schema cache * [Fix] fix lint error * [revert] revert comment format * [revert] revert comment format * use sync.Once instead of atomic.Uint64 * revert comment format --------- Co-authored-by: gunli --- pulsar/producer_partition.go | 21 ++++++++------------- pulsar/schema.go | 17 ++++++++++++----- 2 files changed, 20 insertions(+), 18 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 9d04427a88..012bfd96a8 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -100,29 +100,24 @@ type partitionProducer struct { } type schemaCache struct { - lock sync.RWMutex - schemas map[uint64][]byte + schemas sync.Map } func newSchemaCache() *schemaCache { - return &schemaCache{ - schemas: make(map[uint64][]byte), - } + return &schemaCache{} } func (s *schemaCache) Put(schema *SchemaInfo, schemaVersion []byte) { - s.lock.Lock() - defer s.lock.Unlock() - key := schema.hash() - s.schemas[key] = schemaVersion + s.schemas.Store(key, schemaVersion) } func (s *schemaCache) Get(schema *SchemaInfo) (schemaVersion []byte) { - s.lock.RLock() - defer s.lock.RUnlock() - - return s.schemas[schema.hash()] + val, ok := s.schemas.Load(schema.hash()) + if !ok { + return nil + } + return val.([]byte) } func newPartitionProducer(client *client, topic string, options *ProducerOptions, partitionIdx int, diff --git a/pulsar/schema.go b/pulsar/schema.go index 0b413d40a6..5c063e32a3 100644 --- a/pulsar/schema.go +++ b/pulsar/schema.go @@ -23,6 +23,7 @@ import ( "fmt" "hash/maphash" "reflect" + "sync" "unsafe" log "github.com/sirupsen/logrus" @@ -66,13 +67,19 @@ type SchemaInfo struct { Schema string Type SchemaType Properties map[string]string + hashVal uint64 + hashOnce sync.Once } -func (s SchemaInfo) hash() uint64 { - h := maphash.Hash{} - h.SetSeed(seed) - h.Write([]byte(s.Schema)) - return h.Sum64() +func (s *SchemaInfo) hash() uint64 { + s.hashOnce.Do(func() { + h := maphash.Hash{} + h.SetSeed(seed) + h.Write([]byte(s.Schema)) + s.hashVal = h.Sum64() + }) + + return s.hashVal } type Schema interface { From aa664716ac0bfb886c756f33ba72376092e8e3a5 Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Wed, 5 Jul 2023 10:20:03 +0800 Subject: [PATCH 065/348] [Fix] return when registerSendOrAckOp() failed (#1045) Master Issue: #1040 ### Motivation fix #1040, return when registerSendOrAckOp() return an error. ### Modifications - modified: pulsar/producer_partition.go --------- Co-authored-by: gunli --- pulsar/producer_partition.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 012bfd96a8..595cab4cbd 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1126,6 +1126,7 @@ func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *Producer } if err := transactionImpl.registerSendOrAckOp(); err != nil { runCallback(callback, nil, msg, err) + return } newCallback = func(id MessageID, producerMessage *ProducerMessage, err error) { runCallback(callback, id, producerMessage, err) From b4d6d588d54d12a67dbb5dd71c5a29043290b159 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 5 Jul 2023 10:23:51 +0800 Subject: [PATCH 066/348] Fix the wrong link in the relesae process (#1050) ### Motivation The link to the `GPG keys to sign release artifacts` is dead. The link of the tag is incorrectly pointed to the pulsar-client-node. --- docs/release-process.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/release-process.md b/docs/release-process.md index aa79f9ea34..e3efe8edc7 100644 --- a/docs/release-process.md +++ b/docs/release-process.md @@ -128,7 +128,7 @@ https://dist.apache.org/repos/dist/dev/pulsar/pulsar-client-go-0.X.0-candidate-1 The tag to be voted upon: v0.X.0 -/~https://github.com/apache/pulsar-client-node/releases/tag/v0.X.0 +/~https://github.com/apache/pulsar-client-go/tree/v0.X.0-candidate-1 SHA-512 checksums: 97bb1000f70011e9a585186590e0688586590e09 apache-pulsar-client-go-0.X.0-src.tar.gz From 8d4513787a25423c988708dff985e9c994545df5 Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Thu, 6 Jul 2023 16:29:15 +0800 Subject: [PATCH 067/348] [Fix][Producer] check if message is nil (#1047) * [Fix][Producer] check if message is nil * add a debug log --------- Co-authored-by: gunli --- pulsar/producer_partition.go | 6 ++++++ pulsar/producer_test.go | 11 +++++++++++ 2 files changed, 17 insertions(+) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 595cab4cbd..03729abb47 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1107,6 +1107,12 @@ func (p *partitionProducer) SendAsync(ctx context.Context, msg *ProducerMessage, func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *ProducerMessage, callback func(MessageID, *ProducerMessage, error), flushImmediately bool) { + if msg == nil { + p.log.Error("Message is nil") + runCallback(callback, nil, msg, newError(InvalidMessage, "Message is nil")) + return + } + // Register transaction operation to transaction and the transaction coordinator. var newCallback func(MessageID, *ProducerMessage, error) var txn *transaction diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 7c3abdc721..fecae8efee 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -108,6 +108,9 @@ func TestSimpleProducer(t *testing.T) { assert.NoError(t, err) assert.NotNil(t, ID) } + + _, err = producer.Send(context.Background(), nil) + assert.NotNil(t, err) } func TestProducerAsyncSend(t *testing.T) { @@ -152,6 +155,14 @@ func TestProducerAsyncSend(t *testing.T) { wg.Wait() assert.Equal(t, 0, errors.Size()) + + wg.Add(1) + producer.SendAsync(context.Background(), nil, func(id MessageID, m *ProducerMessage, e error) { + assert.NotNil(t, e) + assert.Nil(t, id) + wg.Done() + }) + wg.Wait() } func TestProducerCompression(t *testing.T) { From 43986b16a90fee6306c95dd23ba30690300d53b5 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Thu, 6 Jul 2023 16:29:56 +0800 Subject: [PATCH 068/348] Add 0.11.0 change log (#1048) --- CHANGELOG.md | 36 ++++++++++++++++++++++++++++++++++++ VERSION | 2 +- stable.txt | 2 +- 3 files changed, 38 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b3e9351a45..973d42f496 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,6 +23,42 @@ All notable changes to this project will be documented in this file. +[0.11.0] 2023-07-04 + +## Features +* Support the schema type ProtoNativeSchema by @gaoran10 in /~https://github.com/apache/pulsar-client-go/pull/1006 +* Implement transactional consumer/producer API by @liangyepianzhou in /~https://github.com/apache/pulsar-client-go/pull/1002 +* Support NonDurable subscriptions by @dinghram in /~https://github.com/apache/pulsar-client-go/pull/992 +* Allow user to specify TLS ciphers an min/max TLS version by @dinghram in /~https://github.com/apache/pulsar-client-go/pull/1041 +* Add single partition router by @crossoverJie in /~https://github.com/apache/pulsar-client-go/pull/999 + +## Improve +* Fix missing link in the release process by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/1000 +* Stablize golangci-lint task in CI by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/1007 +* Fix reconnection backoff logic by @wolfstudy in /~https://github.com/apache/pulsar-client-go/pull/1008 +* Change token name to `GITHUB_TOKEN` in CI by @labuladong in /~https://github.com/apache/pulsar-client-go/pull/910 +* Add links to client docs and feature matrix in README.md by @momo-jun in /~https://github.com/apache/pulsar-client-go/pull/1014 +* Fix flaky test `TestMaxPendingChunkMessages` by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/1003 +* Fix flaky test in `negative_acks_tracker_test.go` by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/1017 +* Fix event time not being set when batching is disabled by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/1015 +* Use maphash instead of crypto/sha256 for hash function of hashmap in Schema.hash() by @bpereto in /~https://github.com/apache/pulsar-client-go/pull/1022 +* Improve logs on failTimeoutMessages by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/1025 +* Delete LICENSE-go-rate.txt by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/1028 +* Fix broken master by upgrading JRE to 17 by @BewareMyPower in /~https://github.com/apache/pulsar-client-go/pull/1030 +* Split sendRequest and make reconnectToBroker and other operation in the same coroutine by @zengguan in /~https://github.com/apache/pulsar-client-go/pull/1029 +* Install openjdk-17 in Dockerfile by @crossoverJie in /~https://github.com/apache/pulsar-client-go/pull/1037 +* Fix ordering key not being set and parsed when batching is disabled by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/1034 +* Check if callback is nil before calling it by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1036 +* Refactor duplicated code lines and fix typo errors by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1039 + +## New Contributors +* @gaoran10 made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1006 +* @momo-jun made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1014 +* @bpereto made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1022 +* @zengguan made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1029 + +**Full Changelog**: /~https://github.com/apache/pulsar-client-go/compare/v0.10.0...v0.11.0-candidate-1 + [0.10.0] 2023-03-27 ## Feature diff --git a/VERSION b/VERSION index 46cd5a5b14..89d00e6236 100644 --- a/VERSION +++ b/VERSION @@ -1,3 +1,3 @@ // This version number refers to the currently released version number // Please fix the version when release. -v0.10.0 +v0.11.0 diff --git a/stable.txt b/stable.txt index fab6099c07..66c47738bd 100644 --- a/stable.txt +++ b/stable.txt @@ -1,3 +1,3 @@ // This version number refers to the current stable version, generally is `VERSION - 1`. // Please fix the version when release. -v0.10.0 +v0.11.0 From dd920ef2670e9b59447540a695db807e7cd9731c Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Mon, 10 Jul 2023 17:10:04 +0800 Subject: [PATCH 069/348] [fix] Fix 0.11.0 change log (#1054) ### Motivation /~https://github.com/apache/pulsar-client-go/pull/1048#issuecomment-1623565815 ### Modifications - Add @gunli to the new contributor list in 0.11.0 change log --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 973d42f496..f1d4839dc2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -56,6 +56,7 @@ All notable changes to this project will be documented in this file. * @momo-jun made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1014 * @bpereto made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1022 * @zengguan made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1029 +* @gunli made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1036 **Full Changelog**: /~https://github.com/apache/pulsar-client-go/compare/v0.10.0...v0.11.0-candidate-1 From be3574019383ac0cdc65fec63e422fcfd6c82e4b Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Tue, 11 Jul 2023 15:35:26 +0800 Subject: [PATCH 070/348] [fix] [issue 877] Fix ctx in partitionProducer.Send() is not performing as expected (#1053) Fixes #877 ### Motivation The original PR is #878. Because the original author @billowqiu has not continued to reply to the review comments for a long time, resubmit the fix here. ### Modifications - Add select for ctx and doneCh in partitionProducer.Send() --------- Co-authored-by: shenjiaqi.2769 --- pulsar/producer_partition.go | 10 +++++++--- pulsar/producer_test.go | 26 ++++++++++++++++++++++++++ 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 03729abb47..dd45ff249b 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1095,9 +1095,13 @@ func (p *partitionProducer) Send(ctx context.Context, msg *ProducerMessage) (Mes }, true) // wait for send request to finish - <-doneCh - - return msgID, err + select { + case <-ctx.Done(): + return nil, ctx.Err() + case <-doneCh: + // send request has been finished + return msgID, err + } } func (p *partitionProducer) SendAsync(ctx context.Context, msg *ProducerMessage, diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index fecae8efee..11ff089387 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -2024,3 +2024,29 @@ func testSendMessagesWithMetadata(t *testing.T, disableBatch bool) { assert.Equal(t, msg.OrderingKey, recvMsg.OrderingKey()) assert.Equal(t, msg.Properties, recvMsg.Properties()) } + +func TestProducerSendWithContext(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + assert.NoError(t, err) + defer client.Close() + + topicName := newTopicName() + // create producer + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topicName, + DisableBatching: true, + }) + assert.Nil(t, err) + defer producer.Close() + + ctx, cancel := context.WithCancel(context.Background()) + // Make ctx be canceled to invalidate the context immediately + cancel() + _, err = producer.Send(ctx, &ProducerMessage{ + Payload: make([]byte, 1024*1024), + }) + // producer.Send should fail and return err context.Canceled + assert.True(t, errors.Is(err, context.Canceled)) +} From e45122c2defc5efd4efc493d0acef278a7ccfc01 Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Thu, 13 Jul 2023 17:15:37 +0800 Subject: [PATCH 071/348] [Fix][Producer] Stop block request even if Value and Payload are both set (#1052) ### Motivation Currently, if `!p.options.DisableBlockIfQueueFull` and `msg.Value != nil && msg.Payload != nil`, request will be blocked forever 'cause `defer request.stopBlock()` is set up after the verify logic. ```go if msg.Value != nil && msg.Payload != nil { p.log.Error("Can not set Value and Payload both") runCallback(request.callback, nil, request.msg, errors.New("can not set Value and Payload both")) return } // The block chan must be closed when returned with exception defer request.stopBlock() ``` Here is the PR to stop block request even if Value and Payload are both set ### Modifications - pulsar/producer_partition.go --------- Co-authored-by: gunli --- pulsar/producer_partition.go | 11 ++++++----- pulsar/producer_test.go | 15 +++++++++++++++ 2 files changed, 21 insertions(+), 5 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index dd45ff249b..48411b4e0c 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -481,11 +481,6 @@ func (p *partitionProducer) internalSend(request *sendRequest) { var schemaPayload []byte var err error - if msg.Value != nil && msg.Payload != nil { - p.log.Error("Can not set Value and Payload both") - runCallback(request.callback, nil, request.msg, errors.New("can not set Value and Payload both")) - return - } // The block chan must be closed when returned with exception defer request.stopBlock() @@ -1117,6 +1112,12 @@ func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *Producer return } + if msg.Value != nil && msg.Payload != nil { + p.log.Error("Can not set Value and Payload both") + runCallback(callback, nil, msg, newError(InvalidMessage, "Can not set Value and Payload both")) + return + } + // Register transaction operation to transaction and the transaction coordinator. var newCallback func(MessageID, *ProducerMessage, error) var txn *transaction diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 11ff089387..adbdc71e67 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -111,6 +111,12 @@ func TestSimpleProducer(t *testing.T) { _, err = producer.Send(context.Background(), nil) assert.NotNil(t, err) + + _, err = producer.Send(context.Background(), &ProducerMessage{ + Payload: []byte("hello"), + Value: []byte("hello"), + }) + assert.NotNil(t, err) } func TestProducerAsyncSend(t *testing.T) { @@ -163,6 +169,15 @@ func TestProducerAsyncSend(t *testing.T) { wg.Done() }) wg.Wait() + + wg.Add(1) + producer.SendAsync(context.Background(), &ProducerMessage{Payload: []byte("hello"), Value: []byte("hello")}, + func(id MessageID, m *ProducerMessage, e error) { + assert.NotNil(t, e) + assert.Nil(t, id) + wg.Done() + }) + wg.Wait() } func TestProducerCompression(t *testing.T) { From 3812c07a0b8f6c37a2803cd740f4bf917160fd22 Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Tue, 18 Jul 2023 18:57:47 +0800 Subject: [PATCH 072/348] [Improve][Producer] simplify the flush logic (#1049) ### Motivation Simplify the producer flush logic ### Modifications 1. add a callback field to the pendingItem, default is nil; 2. in partitionProducer.internalFlush() get the last pendingItem from pendingQueue; 3. update the last pendingItem by setup a new callback; 4. in partitionProducer.ReceivedSendReceipt, no need to identify the sendRequest by checking if the msg is nil; 5. in pendingItem.Complete(), invoke its callback to notify the flush is done. --------- Co-authored-by: gunli --- pulsar/producer_partition.go | 59 +++++++++++++++--------------------- 1 file changed, 24 insertions(+), 35 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 48411b4e0c..e74fd984f4 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -848,11 +848,12 @@ func (p *partitionProducer) internalSingleSend(mm *pb.MessageMetadata, type pendingItem struct { sync.Mutex - buffer internal.Buffer - sequenceID uint64 - sentAt time.Time - sendRequests []interface{} - completed bool + buffer internal.Buffer + sequenceID uint64 + sentAt time.Time + sendRequests []interface{} + completed bool + flushCallback func(err error) } func (p *partitionProducer) internalFlushCurrentBatch() { @@ -990,7 +991,7 @@ func (p *partitionProducer) failTimeoutMessages() { } // flag the sending has completed with error, flush make no effect - pi.Complete() + pi.Complete(errSendTimeout) pi.Unlock() // finally reached the last view item, current iteration ends @@ -1062,15 +1063,10 @@ func (p *partitionProducer) internalFlush(fr *flushRequest) { return } - sendReq := &sendRequest{ - msg: nil, - callback: func(id MessageID, message *ProducerMessage, e error) { - fr.err = e - close(fr.doneCh) - }, + pi.flushCallback = func(err error) { + fr.err = err + close(fr.doneCh) } - - pi.sendRequests = append(pi.sendRequests, sendReq) } func (p *partitionProducer) Send(ctx context.Context, msg *ProducerMessage) (MessageID, error) { @@ -1208,27 +1204,17 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) pi.Lock() defer pi.Unlock() p.metrics.PublishRPCLatency.Observe(float64(now-pi.sentAt.UnixNano()) / 1.0e9) - batchSize := int32(0) - for _, i := range pi.sendRequests { - sr := i.(*sendRequest) - if sr.msg != nil { - batchSize = batchSize + 1 - } else { // Flush request - break - } - } + batchSize := int32(len(pi.sendRequests)) for idx, i := range pi.sendRequests { sr := i.(*sendRequest) - if sr.msg != nil { - atomic.StoreInt64(&p.lastSequenceID, int64(pi.sequenceID)) - p.releaseSemaphoreAndMem(int64(len(sr.msg.Payload))) - p.metrics.PublishLatency.Observe(float64(now-sr.publishTime.UnixNano()) / 1.0e9) - p.metrics.MessagesPublished.Inc() - p.metrics.MessagesPending.Dec() - payloadSize := float64(len(sr.msg.Payload)) - p.metrics.BytesPublished.Add(payloadSize) - p.metrics.BytesPending.Sub(payloadSize) - } + atomic.StoreInt64(&p.lastSequenceID, int64(pi.sequenceID)) + p.releaseSemaphoreAndMem(int64(len(sr.msg.Payload))) + p.metrics.PublishLatency.Observe(float64(now-sr.publishTime.UnixNano()) / 1.0e9) + p.metrics.MessagesPublished.Inc() + p.metrics.MessagesPending.Dec() + payloadSize := float64(len(sr.msg.Payload)) + p.metrics.BytesPublished.Add(payloadSize) + p.metrics.BytesPending.Sub(payloadSize) if sr.callback != nil || len(p.options.Interceptors) > 0 { msgID := newMessageID( @@ -1274,7 +1260,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) } // Mark this pending item as done - pi.Complete() + pi.Complete(nil) } } @@ -1384,12 +1370,15 @@ type flushRequest struct { err error } -func (i *pendingItem) Complete() { +func (i *pendingItem) Complete(err error) { if i.completed { return } i.completed = true buffersPool.Put(i.buffer) + if i.flushCallback != nil { + i.flushCallback(err) + } } // _setConn sets the internal connection field of this partition producer atomically. From 28f61d2f9e00279028e4c6e3558e5aa2665d25cd Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Thu, 20 Jul 2023 15:51:41 +0800 Subject: [PATCH 073/348] [fix] [issue 1051]: Fix inaccurate producer mem limit in chunking and schema (#1055) ### Motivation The producer memory limit have some problem when `EnableChunking=true` or `Schema` is set. - When `Schema` is set, the actual message payload is `msg.Value`. The `len(msg.Payload)` may be 0 and memory can not be reserved acurate. /~https://github.com/apache/pulsar-client-go/blob/be3574019383ac0cdc65fec63e422fcfd6c82e4b/pulsar/producer_partition.go#L479-L494 - In chunking, if producer meets the memory limit, it should release the memory for **chunks which has send out**. But the calculate for this release is not accurate, it should be `uncompressedPayloadSize - int64(lhs)` instead of `uncompressedPayloadSize - int64(rhs)` /~https://github.com/apache/pulsar-client-go/blob/be3574019383ac0cdc65fec63e422fcfd6c82e4b/pulsar/producer_partition.go#L662-L664 - In chunking, if `internalSingleSend` is failed, it should release the memory for **single chunk**. But we release all the chunks memory repeatly now. /~https://github.com/apache/pulsar-client-go/blob/be3574019383ac0cdc65fec63e422fcfd6c82e4b/pulsar/producer_partition.go#L838-L843 - When producer received the receipt from broker, it should release the memory **it reserved before sending**. But it releases wrong size in `chunking` and `schema`. /~https://github.com/apache/pulsar-client-go/blob/be3574019383ac0cdc65fec63e422fcfd6c82e4b/pulsar/producer_partition.go#L1221-L1230 ### Modifications - Fix all the memory limit problems relative to `chunking` and `schema` - Add unit tests to cover these scenarios --------- Co-authored-by: shenjiaqi.2769 --- pulsar/producer_partition.go | 55 +++++++----- pulsar/producer_test.go | 159 +++++++++++++++++++++++++++++++++++ 2 files changed, 192 insertions(+), 22 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index e74fd984f4..5daf54c44b 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -477,21 +477,19 @@ func (p *partitionProducer) internalSend(request *sendRequest) { // read payload from message uncompressedPayload := msg.Payload - uncompressedPayloadSize := int64(len(uncompressedPayload)) var schemaPayload []byte var err error // The block chan must be closed when returned with exception defer request.stopBlock() - if !p.canAddToQueue(request, uncompressedPayloadSize) { + if !p.canAddToQueue(request) { return } if p.options.DisableMultiSchema { if msg.Schema != nil && p.options.Schema != nil && msg.Schema.GetSchemaInfo().hash() != p.options.Schema.GetSchemaInfo().hash() { - p.releaseSemaphoreAndMem(uncompressedPayloadSize) runCallback(request.callback, nil, request.msg, fmt.Errorf("msg schema can not match with producer schema")) p.log.WithError(err).Errorf("The producer %s of the topic %s is disabled the `MultiSchema`", p.producerName, p.topic) return @@ -510,7 +508,6 @@ func (p *partitionProducer) internalSend(request *sendRequest) { if uncompressedPayload == nil && schema != nil { schemaPayload, err = schema.Encode(msg.Value) if err != nil { - p.releaseSemaphoreAndMem(uncompressedPayloadSize) runCallback(request.callback, nil, request.msg, newError(SchemaFailure, err.Error())) p.log.WithError(err).Errorf("Schema encode message failed %s", msg.Value) return @@ -526,7 +523,6 @@ func (p *partitionProducer) internalSend(request *sendRequest) { if schemaVersion == nil { schemaVersion, err = p.getOrCreateSchema(schema.GetSchemaInfo()) if err != nil { - p.releaseSemaphoreAndMem(uncompressedPayloadSize) p.log.WithError(err).Error("get schema version fail") runCallback(request.callback, nil, request.msg, fmt.Errorf("get schema version fail, err: %w", err)) return @@ -537,6 +533,11 @@ func (p *partitionProducer) internalSend(request *sendRequest) { uncompressedSize := len(uncompressedPayload) + // try to reserve memory for uncompressedPayload + if !p.canReserveMem(request, int64(uncompressedSize)) { + return + } + deliverAt := msg.DeliverAt if msg.DeliverAfter.Nanoseconds() > 0 { deliverAt = time.Now().Add(msg.DeliverAfter) @@ -586,7 +587,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { // if msg is too large and chunking is disabled if checkSize > maxMessageSize && !p.options.EnableChunking { - p.releaseSemaphoreAndMem(uncompressedPayloadSize) + p.releaseSemaphoreAndMem(int64(uncompressedSize)) runCallback(request.callback, nil, request.msg, errMessageTooLarge) p.log.WithError(errMessageTooLarge). WithField("size", checkSize). @@ -605,7 +606,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { } else { payloadChunkSize = int(p._getConn().GetMaxMessageSize()) - proto.Size(mm) if payloadChunkSize <= 0 { - p.releaseSemaphoreAndMem(uncompressedPayloadSize) + p.releaseSemaphoreAndMem(int64(uncompressedSize)) runCallback(request.callback, nil, msg, errMetaTooLarge) p.log.WithError(errMetaTooLarge). WithField("metadata size", proto.Size(mm)). @@ -652,10 +653,11 @@ func (p *partitionProducer) internalSend(request *sendRequest) { uuid: uuid, chunkRecorder: cr, transaction: request.transaction, + reservedMem: int64(rhs - lhs), } // the permit of first chunk has acquired - if chunkID != 0 && !p.canAddToQueue(nsr, 0) { - p.releaseSemaphoreAndMem(uncompressedPayloadSize - int64(rhs)) + if chunkID != 0 && !p.canAddToQueue(nsr) { + p.releaseSemaphoreAndMem(int64(uncompressedSize - lhs)) return } p.internalSingleSend(mm, compressedPayload[lhs:rhs], nsr, uint32(maxMessageSize)) @@ -680,7 +682,7 @@ func (p *partitionProducer) internalSend(request *sendRequest) { // after flushing try again to add the current payload if ok := addRequestToBatch(smm, p, uncompressedPayload, request, msg, deliverAt, schemaVersion, multiSchemaEnabled); !ok { - p.releaseSemaphoreAndMem(uncompressedPayloadSize) + p.releaseSemaphoreAndMem(int64(uncompressedSize)) runCallback(request.callback, nil, request.msg, errFailAddToBatch) p.log.WithField("size", uncompressedSize). WithField("properties", msg.Properties). @@ -832,7 +834,7 @@ func (p *partitionProducer) internalSingleSend(mm *pb.MessageMetadata, if err != nil { runCallback(request.callback, nil, request.msg, err) - p.releaseSemaphoreAndMem(int64(len(msg.Payload))) + p.releaseSemaphoreAndMem(request.reservedMem) p.log.WithError(err).Errorf("Single message serialize failed %s", msg.Value) return } @@ -971,7 +973,7 @@ func (p *partitionProducer) failTimeoutMessages() { sr := i.(*sendRequest) if sr.msg != nil { size := len(sr.msg.Payload) - p.releaseSemaphoreAndMem(int64(size)) + p.releaseSemaphoreAndMem(sr.reservedMem) p.metrics.MessagesPending.Dec() p.metrics.BytesPending.Sub(float64(size)) p.metrics.PublishErrorsTimeout.Inc() @@ -1208,7 +1210,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) for idx, i := range pi.sendRequests { sr := i.(*sendRequest) atomic.StoreInt64(&p.lastSequenceID, int64(pi.sequenceID)) - p.releaseSemaphoreAndMem(int64(len(sr.msg.Payload))) + p.releaseSemaphoreAndMem(sr.reservedMem) p.metrics.PublishLatency.Observe(float64(now-sr.publishTime.UnixNano()) / 1.0e9) p.metrics.MessagesPublished.Inc() p.metrics.MessagesPending.Dec() @@ -1352,6 +1354,7 @@ type sendRequest struct { uuid string chunkRecorder *chunkRecorder transaction *transaction + reservedMem int64 } // stopBlock can be invoked multiple times safety @@ -1401,31 +1404,39 @@ func (p *partitionProducer) releaseSemaphoreAndMem(size int64) { p.client.memLimit.ReleaseMemory(size) } -func (p *partitionProducer) canAddToQueue(sr *sendRequest, uncompressedPayloadSize int64) bool { +func (p *partitionProducer) canAddToQueue(sr *sendRequest) bool { if p.options.DisableBlockIfQueueFull { if !p.publishSemaphore.TryAcquire() { runCallback(sr.callback, nil, sr.msg, errSendQueueIsFull) return false } - if !p.client.memLimit.TryReserveMemory(uncompressedPayloadSize) { + } else { + if !p.publishSemaphore.Acquire(sr.ctx) { + runCallback(sr.callback, nil, sr.msg, errContextExpired) + return false + } + } + p.metrics.MessagesPending.Inc() + return true +} + +func (p *partitionProducer) canReserveMem(sr *sendRequest, size int64) bool { + if p.options.DisableBlockIfQueueFull { + if !p.client.memLimit.TryReserveMemory(size) { p.publishSemaphore.Release() runCallback(sr.callback, nil, sr.msg, errMemoryBufferIsFull) return false } } else { - if !p.publishSemaphore.Acquire(sr.ctx) { - runCallback(sr.callback, nil, sr.msg, errContextExpired) - return false - } - if !p.client.memLimit.ReserveMemory(sr.ctx, uncompressedPayloadSize) { + if !p.client.memLimit.ReserveMemory(sr.ctx, size) { p.publishSemaphore.Release() runCallback(sr.callback, nil, sr.msg, errContextExpired) return false } } - p.metrics.MessagesPending.Inc() - p.metrics.BytesPending.Add(float64(len(sr.msg.Payload))) + sr.reservedMem += size + p.metrics.BytesPending.Add(float64(size)) return true } diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index adbdc71e67..be9885fa03 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -1939,6 +1939,165 @@ func TestMemLimitRejectProducerMessages(t *testing.T) { assert.NoError(t, err) } +func TestMemLimitRejectProducerMessagesWithSchema(t *testing.T) { + + c, err := NewClient(ClientOptions{ + URL: serviceURL, + MemoryLimitBytes: 100 * 6, + }) + assert.NoError(t, err) + defer c.Close() + + schema := NewAvroSchema(`{"fields": + [ + {"name":"id","type":"int"},{"default":null,"name":"name","type":["null","string"]} + ], + "name":"MyAvro","namespace":"schemaNotFoundTestCase","type":"record"}`, nil) + + topicName := newTopicName() + producer1, _ := c.CreateProducer(ProducerOptions{ + Topic: topicName, + DisableBlockIfQueueFull: true, + DisableBatching: false, + BatchingMaxPublishDelay: 100 * time.Second, + SendTimeout: 2 * time.Second, + }) + + producer2, _ := c.CreateProducer(ProducerOptions{ + Topic: topicName, + DisableBlockIfQueueFull: true, + DisableBatching: false, + BatchingMaxPublishDelay: 100 * time.Second, + SendTimeout: 2 * time.Second, + }) + + // the size of encoded value is 6 bytes + value := map[string]interface{}{ + "id": 0, + "name": map[string]interface{}{ + "string": "abc", + }, + } + + n := 101 + for i := 0; i < n/2; i++ { + producer1.SendAsync(context.Background(), &ProducerMessage{ + Value: value, + Schema: schema, + }, func(id MessageID, message *ProducerMessage, e error) {}) + + producer2.SendAsync(context.Background(), &ProducerMessage{ + Value: value, + Schema: schema, + }, func(id MessageID, message *ProducerMessage, e error) {}) + } + // Last message in order to reach the limit + producer1.SendAsync(context.Background(), &ProducerMessage{ + Value: value, + Schema: schema, + }, func(id MessageID, message *ProducerMessage, e error) {}) + time.Sleep(100 * time.Millisecond) + assert.Equal(t, int64(n*6), c.(*client).memLimit.CurrentUsage()) + + _, err = producer1.Send(context.Background(), &ProducerMessage{ + Value: value, + Schema: schema, + }) + assert.Error(t, err) + assert.ErrorContains(t, err, getResultStr(ClientMemoryBufferIsFull)) + + _, err = producer2.Send(context.Background(), &ProducerMessage{ + Value: value, + Schema: schema, + }) + assert.Error(t, err) + assert.ErrorContains(t, err, getResultStr(ClientMemoryBufferIsFull)) + + // flush pending msg + err = producer1.Flush() + assert.NoError(t, err) + err = producer2.Flush() + assert.NoError(t, err) + assert.Equal(t, int64(0), c.(*client).memLimit.CurrentUsage()) + + _, err = producer1.Send(context.Background(), &ProducerMessage{ + Value: value, + Schema: schema, + }) + assert.NoError(t, err) + _, err = producer2.Send(context.Background(), &ProducerMessage{ + Value: value, + Schema: schema, + }) + assert.NoError(t, err) +} + +func TestMemLimitRejectProducerMessagesWithChunking(t *testing.T) { + + c, err := NewClient(ClientOptions{ + URL: serviceURL, + MemoryLimitBytes: 5 * 1024, + }) + assert.NoError(t, err) + defer c.Close() + + topicName := newTopicName() + producer1, _ := c.CreateProducer(ProducerOptions{ + Topic: topicName, + DisableBlockIfQueueFull: true, + DisableBatching: true, + EnableChunking: true, + SendTimeout: 2 * time.Second, + }) + + producer2, _ := c.CreateProducer(ProducerOptions{ + Topic: topicName, + DisableBlockIfQueueFull: true, + DisableBatching: false, + BatchingMaxPublishDelay: 100 * time.Millisecond, + SendTimeout: 2 * time.Second, + }) + + producer2.SendAsync(context.Background(), &ProducerMessage{ + Payload: make([]byte, 5*1024+1), + }, func(id MessageID, message *ProducerMessage, e error) { + if e != nil { + t.Fatal(e) + } + }) + + time.Sleep(50 * time.Millisecond) + assert.Equal(t, int64(5*1024+1), c.(*client).memLimit.CurrentUsage()) + + _, err = producer1.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1), + }) + assert.Error(t, err) + assert.ErrorContains(t, err, getResultStr(ClientMemoryBufferIsFull)) + + // wait all the mem have been released + retryAssert(t, 10, 200, func() {}, func(t assert.TestingT) bool { + return assert.Equal(t, 0, int(c.(*client).memLimit.CurrentUsage())) + }) + + producer3, _ := c.CreateProducer(ProducerOptions{ + Topic: topicName, + DisableBlockIfQueueFull: true, + DisableBatching: true, + EnableChunking: true, + MaxPendingMessages: 1, + ChunkMaxMessageSize: 1024, + SendTimeout: 2 * time.Second, + }) + + // producer2 will reserve 2*1024 bytes and then release 1024 byte (release the second chunk) + // because it reaches MaxPendingMessages in chunking + _, _ = producer3.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 2*1024), + }) + assert.Equal(t, int64(1024), c.(*client).memLimit.CurrentUsage()) +} + func TestMemLimitContextCancel(t *testing.T) { c, err := NewClient(ClientOptions{ From d4e08c699632b19aef70011cf6c3671e739bc101 Mon Sep 17 00:00:00 2001 From: Michael Marshall Date: Thu, 20 Jul 2023 05:20:05 -0500 Subject: [PATCH 074/348] [fix] Send Close Command on Producer/Consumer create timeout (#1061) ### Motivation This change is the same as /~https://github.com/apache/pulsar/pull/13161 and /~https://github.com/apache/pulsar/pull/16616, and is justified by these lines of our binary protocol spec: * /~https://github.com/apache/pulsar-site/blob/9b4b3d39014bd47c0bb9f66742b89bcb40ed7f07/docs/developing-binary-protocol.md?plain=1#L301-L304 * /~https://github.com/apache/pulsar-site/blob/9b4b3d39014bd47c0bb9f66742b89bcb40ed7f07/docs/developing-binary-protocol.md?plain=1#L468-L471 ### Modifications * When a producer or a consumer times out during creation, make an attempt to close the producer or consumer by sending the appropriate close command. Failures can safely be ignored because the only time that the close will actually matter is when the TCP connection is open for other protocol messages. The one nuance is that we send the close command to the same address pair that we send the create command. --- pulsar/consumer_partition.go | 9 +++++++++ pulsar/producer_partition.go | 8 ++++++++ 2 files changed, 17 insertions(+) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index cd7aa50199..28cb9c5567 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -1734,6 +1734,15 @@ func (pc *partitionConsumer) grabConn() error { if err != nil { pc.log.WithError(err).Error("Failed to create consumer") + if err == internal.ErrRequestTimeOut { + requestID := pc.client.rpcClient.NewRequestID() + cmdClose := &pb.CommandCloseConsumer{ + ConsumerId: proto.Uint64(pc.consumerID), + RequestId: proto.Uint64(requestID), + } + _, _ = pc.client.rpcClient.Request(lr.LogicalAddr, lr.PhysicalAddr, requestID, + pb.BaseCommand_CLOSE_CONSUMER, cmdClose) + } return err } diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 5daf54c44b..7d514d550e 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -251,6 +251,14 @@ func (p *partitionProducer) grabCnx() error { res, err := p.client.rpcClient.Request(lr.LogicalAddr, lr.PhysicalAddr, id, pb.BaseCommand_PRODUCER, cmdProducer) if err != nil { p.log.WithError(err).Error("Failed to create producer at send PRODUCER request") + if err == internal.ErrRequestTimeOut { + id := p.client.rpcClient.NewRequestID() + _, _ = p.client.rpcClient.Request(lr.LogicalAddr, lr.PhysicalAddr, id, pb.BaseCommand_CLOSE_PRODUCER, + &pb.CommandCloseProducer{ + ProducerId: &p.producerID, + RequestId: &id, + }) + } return err } From 9867c29ca329302e97ddd9c6a99f66853c7f447f Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Thu, 20 Jul 2023 19:35:05 +0800 Subject: [PATCH 075/348] [fix] [issue 1057]: Fix the producer flush opertion is not guarantee to flush all messages (#1058) Fixes #1057 ### Motivation `dataChan` is introduced by #1029 to fix the problem of reconnectToBroker. But it missed that if a flush operation excuted, there may still be some messages in `dataChan`. And these messages can't be flushed. ### Modifications - Fix the producer flush opertion is not guarantee to flush all messages --- pulsar/producer_partition.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 7d514d550e..11d5f652e4 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1051,6 +1051,15 @@ func (p *partitionProducer) internalFlushCurrentBatches() { } func (p *partitionProducer) internalFlush(fr *flushRequest) { + // clear all the messages which have sent to dataChan before flush + if len(p.dataChan) != 0 { + oldDataChan := p.dataChan + p.dataChan = make(chan *sendRequest, p.options.MaxPendingMessages) + for len(oldDataChan) != 0 { + pendingData := <-oldDataChan + p.internalSend(pendingData) + } + } p.internalFlushCurrentBatch() From 4bfd4aae74efa83fd43400371731258c0afb5d74 Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Tue, 25 Jul 2023 18:39:51 +0800 Subject: [PATCH 076/348] [fix] [issue 1064] Fix the panic when try to flush in DisableBatching=true (#1065) Fixes #1064 ### Motivation If we set producer `DisableBatching=true`, it will be panic when call `producer.Flush()`. More details in #1064 . ### Modifications - Avoid panic in non-batching producer - Add unit test to cover `Flush()` in non-batching producer. --- pulsar/producer_partition.go | 4 +++- pulsar/producer_test.go | 44 ++++++++++++++++++++++++++++++++++++ 2 files changed, 47 insertions(+), 1 deletion(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 11d5f652e4..6bdb0905d4 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1061,7 +1061,9 @@ func (p *partitionProducer) internalFlush(fr *flushRequest) { } } - p.internalFlushCurrentBatch() + if !p.options.DisableBatching { + p.internalFlushCurrentBatch() + } pi, ok := p.pendingQueue.PeekLast().(*pendingItem) if !ok { diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index be9885fa03..a9a7b81911 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -180,6 +180,50 @@ func TestProducerAsyncSend(t *testing.T) { wg.Wait() } +func TestProducerFlushDisableBatching(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: serviceURL, + }) + assert.NoError(t, err) + defer client.Close() + + producer, err := client.CreateProducer(ProducerOptions{ + Topic: newTopicName(), + DisableBatching: true, + }) + + assert.NoError(t, err) + assert.NotNil(t, producer) + defer producer.Close() + + wg := sync.WaitGroup{} + wg.Add(10) + errors := internal.NewBlockingQueue(10) + + for i := 0; i < 10; i++ { + producer.SendAsync(context.Background(), &ProducerMessage{ + Payload: []byte("hello"), + }, func(id MessageID, message *ProducerMessage, e error) { + if e != nil { + log.WithError(e).Error("Failed to publish") + errors.Put(e) + } else { + log.Info("Published message ", id) + } + wg.Done() + }) + + assert.NoError(t, err) + } + + err = producer.Flush() + assert.Nil(t, err) + + wg.Wait() + + assert.Equal(t, 0, errors.Size()) +} + func TestProducerCompression(t *testing.T) { type testProvider struct { name string From 59ef32b03f7595a905388f25b34caaf8ebecffb2 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Wed, 26 Jul 2023 18:57:56 +0800 Subject: [PATCH 077/348] [fix][txn] Fix the transaction acknowledgement and send logic for chunk message (#1069) Master /~https://github.com/apache/pulsar-client-go/issues/1060 ### Motivation 1. For the chunk message, we only register the send operation once but end the send operation multiple times when receiving the send response. It will make the transaction can be committed before all the operations are completed. 2. When we use transaction ack for chunk messages, the provided transaction is ignored, resulting in the chunk message actually being acknowledged using the non-transactional ack method. ### Modifications 1. Only end the send operation when receive the last chunk message. 2. Add the check for the transaction when the massage is a chunk message. --- pulsar/consumer_partition.go | 17 +++++- pulsar/transaction_test.go | 101 ++++++++++++++++++++++++++++++++++- 2 files changed, 114 insertions(+), 4 deletions(-) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 28cb9c5567..b50c0a0019 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -426,7 +426,10 @@ func (pc *partitionConsumer) ackIDCommon(msgID MessageID, withResponse bool, txn } if cmid, ok := msgID.(*chunkMessageID); ok { - return pc.unAckChunksTracker.ack(cmid) + if txn == nil { + return pc.unAckChunksTracker.ack(cmid) + } + return pc.unAckChunksTracker.ackWithTxn(cmid, txn) } trackingID := toTrackingMessageID(msgID) @@ -2212,9 +2215,19 @@ func (u *unAckChunksTracker) remove(cmid *chunkMessageID) { } func (u *unAckChunksTracker) ack(cmid *chunkMessageID) error { + return u.ackWithTxn(cmid, nil) +} + +func (u *unAckChunksTracker) ackWithTxn(cmid *chunkMessageID, txn Transaction) error { ids := u.get(cmid) for _, id := range ids { - if err := u.pc.AckID(id); err != nil { + var err error + if txn == nil { + err = u.pc.AckID(id) + } else { + err = u.pc.AckIDWithTxn(id, txn) + } + if err != nil { return err } } diff --git a/pulsar/transaction_test.go b/pulsar/transaction_test.go index 66a82ccad1..385b197e00 100644 --- a/pulsar/transaction_test.go +++ b/pulsar/transaction_test.go @@ -423,6 +423,14 @@ func TestTransactionAbort(t *testing.T) { // Abort the transaction. _ = txn.Abort(context.Background()) + consumerShouldNotReceiveMessage(t, consumer) + + // Clean up: Close the consumer and producer instances. + consumer.Close() + producer.Close() +} + +func consumerShouldNotReceiveMessage(t *testing.T, consumer Consumer) { // Expectation: The consumer should not receive any messages. done := make(chan struct{}) go func() { @@ -438,8 +446,97 @@ func TestTransactionAbort(t *testing.T) { require.Fail(t, "The consumer should not receive any messages") case <-time.After(time.Second): } +} - // Clean up: Close the consumer and producer instances. +func TestAckChunkMessage(t *testing.T) { + topic := newTopicName() + sub := "my-sub" + + // Prepare: Create PulsarClient and initialize the transaction coordinator client. + _, client := createTcClient(t) + + // Create transaction and register the send operation. + txn, err := client.NewTransaction(time.Hour) + require.Nil(t, err) + txn.(*transaction).registerSendOrAckOp() + + // Create a producer with chunking enabled to send a large message that will be split into chunks. + producer, err := client.CreateProducer(ProducerOptions{ + Name: "test", + Topic: topic, + EnableChunking: true, + DisableBatching: true, + }) + require.NoError(t, err) + require.NotNil(t, producer) + defer producer.Close() + + // Subscribe to the consumer. + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + Type: Exclusive, + SubscriptionName: sub, + }) + require.NoError(t, err) + defer consumer.Close() + + // Send a large message that will be split into chunks. + msgID, err := producer.Send(context.Background(), &ProducerMessage{ + Transaction: txn, + Payload: createTestMessagePayload(_brokerMaxMessageSize), + }) + require.NoError(t, err) + _, ok := msgID.(*chunkMessageID) + require.True(t, ok) + + err = txn.Commit(context.Background()) + require.Nil(t, err) + + // Receive the message using a new transaction and ack it. + txn2, err := client.NewTransaction(time.Hour) + require.Nil(t, err) + message, err := consumer.Receive(context.Background()) + require.Nil(t, err) + + err = consumer.AckWithTxn(message, txn2) + require.Nil(t, err) + + txn2.Abort(context.Background()) + + // Close the consumer to simulate reconnection and receive the same message again. consumer.Close() - producer.Close() + + // Subscribe to the consumer again. + consumer, err = client.Subscribe(ConsumerOptions{ + Topic: topic, + Type: Exclusive, + SubscriptionName: sub, + }) + require.Nil(t, err) + message, err = consumer.Receive(context.Background()) + require.Nil(t, err) + require.NotNil(t, message) + + // Create a new transaction and ack the message again. + txn3, err := client.NewTransaction(time.Hour) + require.Nil(t, err) + + err = consumer.AckWithTxn(message, txn3) + require.Nil(t, err) + + // Commit the third transaction. + err = txn3.Commit(context.Background()) + require.Nil(t, err) + + // Close the consumer again. + consumer.Close() + + // Subscribe to the consumer again and verify that no message is received. + consumer, err = client.Subscribe(ConsumerOptions{ + Topic: topic, + Type: Exclusive, + SubscriptionName: sub, + }) + require.Nil(t, err) + consumerShouldNotReceiveMessage(t, consumer) } From a3fcc9a5422c415e0124c3efbf12ecb6596c12ef Mon Sep 17 00:00:00 2001 From: Michael Marshall Date: Wed, 26 Jul 2023 06:09:21 -0500 Subject: [PATCH 078/348] [fix] Close consumer resources if creation fails (#1070) ### Motivation When a consumer fails to get created, we should close any resources that it created to prevent leaks of internal resources and leaks of the consumer on the broker side. The broker leak could happen if the connection was left open. These fixes are similar to #1061. ### Modifications * Close `ackGroupingTracker` and `chunkedMsgCtxMap` if `grabConn` fails. We cannot call `Close` on the consumer because the state is not `Ready`. If we re-design the consumer, it could be nice to be able to call `Close` in this scenario. * Call `Close` on the consumer in cases where we move it to `Ready` but determine it is not able to be created. * Fix typo in comment --- pulsar/consumer_partition.go | 6 ++++-- pulsar/producer_partition.go | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index b50c0a0019..364dae754c 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -372,6 +372,8 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon if err != nil { pc.log.WithError(err).Error("Failed to create consumer") pc.nackTracker.Close() + pc.ackGroupingTracker.close() + pc.chunkedMsgCtxMap.Close() return nil, err } pc.log.Info("Created consumer") @@ -381,7 +383,7 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon if pc.options.startMessageIDInclusive && startingMessageID != nil && startingMessageID.equal(latestMessageID) { msgID, err := pc.requestGetLastMessageID() if err != nil { - pc.nackTracker.Close() + pc.Close() return nil, err } if msgID.entryID != noMessageEntry { @@ -390,7 +392,7 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon // use the WithoutClear version because the dispatcher is not started yet err = pc.requestSeekWithoutClear(msgID.messageID) if err != nil { - pc.nackTracker.Close() + pc.Close() return nil, err } } diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 6bdb0905d4..d3f61ef101 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1340,7 +1340,7 @@ func (p *partitionProducer) setProducerState(state producerState) { p.state.Swap(int32(state)) } -// set a new consumerState and return the last state +// set a new producerState and return the last state // returns bool if the new state has been set or not func (p *partitionProducer) casProducerState(oldState, newState producerState) bool { return p.state.CAS(int32(oldState), int32(newState)) From 16a029932d8613a08b542547488e97d321e771a4 Mon Sep 17 00:00:00 2001 From: Jorge Pereira Date: Thu, 27 Jul 2023 12:07:37 +0100 Subject: [PATCH 079/348] Fix: client reconnected every authenticationRefreshCheckSeconds when using tls authentication (#1062) ### Motivation When using pulsar tls authentication with a broker that sets the authenticationRefreshCheckSeconds the connection was dropped for each authentication refresh check. After analyzing logs and tcpdumps I concluded that this error appears because the tls authentication is returning null, witch does not pass a validation in the broker. After analyzing the tls auth implementation in Java (that works), I concluded that the GetData method should return empty byte array instead of nil. ### Modifications Changed tls auth GetData to return empty byte array instead of nil. --------- Co-authored-by: Jorge Pereira --- pulsar/internal/connection.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 1f80d20d33..e2ae7ac845 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -823,6 +823,11 @@ func (c *connection) handleAuthChallenge(authChallenge *pb.CommandAuthChallenge) return } + // Brokers expect authData to be not nil + if authData == nil { + authData = []byte{} + } + cmdAuthResponse := &pb.CommandAuthResponse{ ProtocolVersion: proto.Int32(PulsarProtocolVersion), ClientVersion: proto.String(ClientVersionString), From 50015d30b807d47d0f7db25d79662817c9f4649e Mon Sep 17 00:00:00 2001 From: Jiaqi Shen <18863662628@163.com> Date: Thu, 27 Jul 2023 19:08:16 +0800 Subject: [PATCH 080/348] [fix] Correct the SendAsync() description (#1066) ### Motivation After #1029 , `eventChan` is split into `dataChan` and `cmdChan`. But the description of `SendAsync()` is not modified. /~https://github.com/apache/pulsar-client-go/blob/9867c29ca329302e97ddd9c6a99f66853c7f447f/pulsar/producer.go#L226-L231 ### Modifications - Correct the description of SendAsync() description --- pulsar/producer.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pulsar/producer.go b/pulsar/producer.go index 8fcb891eb8..70d152c78b 100644 --- a/pulsar/producer.go +++ b/pulsar/producer.go @@ -224,8 +224,7 @@ type Producer interface { Send(context.Context, *ProducerMessage) (MessageID, error) // SendAsync a message in asynchronous mode - // This call is blocked when the `event channel` becomes full (default: 10) or the - // `maxPendingMessages` becomes full (default: 1000) + // This call is blocked when the `maxPendingMessages` becomes full (default: 1000) // The callback will report back the message being published and // the eventual error in publishing SendAsync(context.Context, *ProducerMessage, func(MessageID, *ProducerMessage, error)) From 8828c0170f1aa327492585444521da372f58b968 Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 15 Aug 2023 23:33:17 +0800 Subject: [PATCH 081/348] ci: replace license header checker and formatter (#1077) This refers to #1075. korandoru/hawkeye can auto-format all the incoming files at once. Signed-off-by: tison --- .asf.yaml | 2 - .github/workflows/ci.yml | 5 +- Dockerfile | 2 - integration-tests/certs/generate.sh | 3 +- integration-tests/license_test.go | 118 ------------------- integration-tests/pb/build.sh | 3 +- integration-tests/pb/hello.proto | 3 +- integration-tests/pb/hi.proto | 19 +++ licenserc.toml | 25 ++++ pulsar/internal/pulsar_proto/PulsarApi.proto | 3 +- scripts/pulsar-test-service-start.sh | 3 +- scripts/pulsar-test-service-stop.sh | 3 +- scripts/run-ci.sh | 3 +- 13 files changed, 57 insertions(+), 135 deletions(-) delete mode 100644 integration-tests/license_test.go create mode 100644 licenserc.toml diff --git a/.asf.yaml b/.asf.yaml index 05e8ef3f2f..246ec81885 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -1,4 +1,3 @@ -# # 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 @@ -15,7 +14,6 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -# github: description: "Apache Pulsar Go Client Library" diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 7c946b7652..f0c9b63999 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -29,7 +29,10 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v3 - - run: make lint + - name: Check license header + run: docker run --rm -v $(pwd):/github/workspace ghcr.io/korandoru/hawkeye-native:v3 check + - name: Run golangci-lint + run: make lint integration-tests: runs-on: ubuntu-latest diff --git a/Dockerfile b/Dockerfile index 07eff8ea2d..1c137fc096 100644 --- a/Dockerfile +++ b/Dockerfile @@ -1,4 +1,3 @@ -# # 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 @@ -15,7 +14,6 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -# # Explicit version of Pulsar and Golang images should be # set via the Makefile or CLI diff --git a/integration-tests/certs/generate.sh b/integration-tests/certs/generate.sh index 9294e0342c..ea365c1d9f 100755 --- a/integration-tests/certs/generate.sh +++ b/integration-tests/certs/generate.sh @@ -1,5 +1,4 @@ #!/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 @@ -16,7 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -# + export CA_HOME=$(pwd) echo $CA_HOME diff --git a/integration-tests/license_test.go b/integration-tests/license_test.go deleted file mode 100644 index ac89eb4dc2..0000000000 --- a/integration-tests/license_test.go +++ /dev/null @@ -1,118 +0,0 @@ -// 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 license_test - -import ( - "os" - "path/filepath" - "regexp" - "testing" -) - -var goFileCheck = regexp.MustCompile(`// 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\. - -`) - -var otherCheck = regexp.MustCompile(`# -# 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\. -# -`) - -var skip = map[string]bool{ - "../pulsar/internal/pulsar_proto/PulsarApi.pb.go": true, - "../.github/workflows/bot.yaml": true, - "../integration-tests/pb/hello.pb.go": true, - "../integration-tests/.htpasswd": true, -} - -func TestLicense(t *testing.T) { - err := filepath.Walk("../.", func(path string, fi os.FileInfo, err error) error { - if skip[path] { - return nil - } - - if err != nil { - return err - } - - switch filepath.Ext(path) { - case ".go": - src, err := os.ReadFile(path) - if err != nil { - return nil - } - - // Find license - if !goFileCheck.Match(src) { - t.Errorf("%v: license header not present", path) - return nil - } - case ".yaml": - fallthrough - case ".conf": - src, err := os.ReadFile(path) - if err != nil { - return nil - } - - // Find license - if !otherCheck.Match(src) { - t.Errorf("%v: license header not present", path) - return nil - } - - default: - return nil - } - - return nil - }) - if err != nil { - t.Fatal(err) - } -} diff --git a/integration-tests/pb/build.sh b/integration-tests/pb/build.sh index e1924581d0..651147287a 100755 --- a/integration-tests/pb/build.sh +++ b/integration-tests/pb/build.sh @@ -1,5 +1,4 @@ #!/usr/bin/env 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 @@ -16,7 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -# + PROJECT_DIR=$(git rev-parse --show-toplevel) SOURCE_PATH="$PROJECT_DIR/integration-tests/pb" diff --git a/integration-tests/pb/hello.proto b/integration-tests/pb/hello.proto index ca6a4ff9e8..5b33a52f5c 100644 --- a/integration-tests/pb/hello.proto +++ b/integration-tests/pb/hello.proto @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + syntax = "proto3"; package prototest; diff --git a/integration-tests/pb/hi.proto b/integration-tests/pb/hi.proto index f5a401a54d..2f50ce31c9 100644 --- a/integration-tests/pb/hi.proto +++ b/integration-tests/pb/hi.proto @@ -1,3 +1,22 @@ +/* + * 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. + */ + syntax = "proto3"; package prototest; diff --git a/licenserc.toml b/licenserc.toml new file mode 100644 index 0000000000..4ca4e14faf --- /dev/null +++ b/licenserc.toml @@ -0,0 +1,25 @@ +# 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. + +headerPath = "Apache-2.0-ASF.txt" + +excludes = [ + "*.txt", +] + +[mapping.DOUBLESLASH_STYLE] +extensions = ["go"] diff --git a/pulsar/internal/pulsar_proto/PulsarApi.proto b/pulsar/internal/pulsar_proto/PulsarApi.proto index 0c72361c7f..8bd75729c8 100644 --- a/pulsar/internal/pulsar_proto/PulsarApi.proto +++ b/pulsar/internal/pulsar_proto/PulsarApi.proto @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + syntax = "proto2"; package pulsar.proto; diff --git a/scripts/pulsar-test-service-start.sh b/scripts/pulsar-test-service-start.sh index ed90b0660a..1f62558a52 100755 --- a/scripts/pulsar-test-service-start.sh +++ b/scripts/pulsar-test-service-start.sh @@ -1,5 +1,4 @@ #!/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 @@ -16,7 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -# + set -e diff --git a/scripts/pulsar-test-service-stop.sh b/scripts/pulsar-test-service-stop.sh index eafccd98f9..fffcd8b3f6 100755 --- a/scripts/pulsar-test-service-stop.sh +++ b/scripts/pulsar-test-service-stop.sh @@ -1,5 +1,4 @@ #!/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 @@ -16,7 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -# + set -e diff --git a/scripts/run-ci.sh b/scripts/run-ci.sh index 8eebdec271..cc4f6a1e74 100755 --- a/scripts/run-ci.sh +++ b/scripts/run-ci.sh @@ -1,5 +1,4 @@ #!/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 @@ -16,7 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -# + set -e -x From c49345ca3a596a7c69a672f5ced341240a5f8df1 Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 16 Aug 2023 02:15:40 +0800 Subject: [PATCH 082/348] chore: allow rebase and merge (#1080) For /~https://github.com/apache/pulsar-client-go/pull/1079. We can toggle the config once that PR gets merged. --- .asf.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index 246ec81885..ddaa0df7d0 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -40,7 +40,7 @@ github: # disable merge button: merge: false # disable rebase button: - rebase: false + rebase: true notifications: From a561477a9c005d6fb02609eca3d6f4ab05fc8ce5 Mon Sep 17 00:00:00 2001 From: StreamNative Bot <44651360+streamnativebot@users.noreply.github.com> Date: Thu, 29 Sep 2022 09:02:09 -0700 Subject: [PATCH 083/348] Create new CODEOWNERS file --- pulsaradmin/.github/CODEOWNERS | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 pulsaradmin/.github/CODEOWNERS diff --git a/pulsaradmin/.github/CODEOWNERS b/pulsaradmin/.github/CODEOWNERS new file mode 100644 index 0000000000..c3875a1248 --- /dev/null +++ b/pulsaradmin/.github/CODEOWNERS @@ -0,0 +1,2 @@ +# Owning team's slack is #f_sn_data_plane +* @streamnative/cloud From 01b97be948d1a1c0d8e9aef4e3679d55c58015b5 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Thu, 11 Jul 2019 22:26:55 -0700 Subject: [PATCH 084/348] Initialize the pulsarctl project (streamnative/pulsarctl#1) * Initialize the pulsarctl project *Modifications* - setup the project layout - add two clusters command --- pulsaradmin/go.mod | 14 ++ pulsaradmin/pkg/pulsar/admin.go | 283 ++++++++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/cluster.go | 48 +++++ pulsaradmin/pkg/pulsar/data.go | 11 ++ pulsaradmin/pkg/pulsar/errors.go | 19 ++ pulsaradmin/pkg/pulsar/utils.go | 7 + 6 files changed, 382 insertions(+) create mode 100644 pulsaradmin/go.mod create mode 100644 pulsaradmin/pkg/pulsar/admin.go create mode 100644 pulsaradmin/pkg/pulsar/cluster.go create mode 100644 pulsaradmin/pkg/pulsar/data.go create mode 100644 pulsaradmin/pkg/pulsar/errors.go create mode 100644 pulsaradmin/pkg/pulsar/utils.go diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod new file mode 100644 index 0000000000..7f2ca09017 --- /dev/null +++ b/pulsaradmin/go.mod @@ -0,0 +1,14 @@ +module github.com/streamnative/pulsar-admin-go + +go 1.12 + +require ( + github.com/fatih/color v1.7.0 // indirect + github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 + github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b // indirect + github.com/mattn/go-colorable v0.1.2 // indirect + github.com/mattn/go-runewidth v0.0.4 // indirect + github.com/olekukonko/tablewriter v0.0.1 + github.com/spf13/cobra v0.0.5 + github.com/spf13/pflag v1.0.3 +) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go new file mode 100644 index 0000000000..a6f5768dd0 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -0,0 +1,283 @@ +package pulsar + +import ( + "bytes" + "encoding/json" + "fmt" + "io" + "io/ioutil" + "net/http" + "net/url" + "path" +) + +const ( + DefaultWebServiceURL = "http://localhost:8080" +) + +// Config is used to configure the admin client +type Config struct { + WebServiceUrl string + HttpClient *http.Client +} + +// DefaultConfig returns a default configuration for the pulsar admin client +func DefaultConfig() *Config { + config := &Config{ + WebServiceUrl: DefaultWebServiceURL, + HttpClient: http.DefaultClient, + } + return config +} + +// Client provides a client to the Pulsar Restful API +type Client interface { + Clusters() Clusters +} + +type client struct { + webServiceUrl string + apiVersion string + httpClient *http.Client +} + +// New returns a new client +func New(config *Config) Client { + defConfig := DefaultConfig() + + if len(config.WebServiceUrl) == 0 { + config.WebServiceUrl = defConfig.WebServiceUrl + } + + c := &client{ + // TODO: make api version configurable + apiVersion: "v2", + webServiceUrl: config.WebServiceUrl, + } + + return c +} + +func (c *client) endpoint(componentPath string, parts ...string) string { + return path.Join(makeHttpPath(c.apiVersion, componentPath), endpoint(parts...)) +} + +// get is used to do a GET request against an endpoint +// and deserialize the response into an interface +func (c *client) get(endpoint string, obj interface{}) error { + req, err := c.newRequest(http.MethodGet, endpoint) + if err != nil { + return err + } + + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + + if obj != nil { + if err := decodeJsonBody(resp, &obj); err != nil { + return err + } + } + + return nil +} + +func (c *client) put(endpoint string, in, obj interface{}) error { + req, err := c.newRequest(http.MethodPut, endpoint) + if err != nil { + return err + } + req.obj = in + + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + + if obj != nil { + if err := decodeJsonBody(resp, &obj); err != nil { + return err + } + } + + return nil +} + +func (c *client) delete(endpoint string, obj interface{}) error { + req, err := c.newRequest(http.MethodDelete, endpoint) + if err != nil { + return err + } + + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + + if obj != nil { + if err := decodeJsonBody(resp, &obj); err != nil { + return err + } + } + + return nil +} + +type request struct { + method string + url *url.URL + params url.Values + + obj interface{} + body io.Reader +} + +func (r *request) toHTTP() (*http.Request, error) { + r.url.RawQuery = r.params.Encode() + + // add a request body if there is one + if r.body == nil && r.obj != nil { + body, err := encodeJsonBody(r.obj) + if err != nil { + return nil, err + } + r.body = body + } + + req, err := http.NewRequest(r.method, r.url.RequestURI(), r.body) + if err != nil { + return nil, err + } + + req.URL.Host = r.url.Host + req.URL.Scheme = r.url.Scheme + req.Host = r.url.Host + return req, nil +} + + +func (c *client) newRequest(method, path string) (*request, error) { + base, _ := url.Parse(c.webServiceUrl) + u, err := url.Parse(path) + if err != nil { + return nil, err + } + req := &request{ + method: method, + url: &url.URL{ + Scheme: base.Scheme, + User: base.User, + Host: base.Host, + Path: endpoint(base.Path, u.Path), + }, + params: make(url.Values), + } + return req, nil +} + +// TODO: add pulsarctl version +func (c *client) useragent() string { + return fmt.Sprintf("pulsarctl (go)") +} + +func (c *client) doRequest(r *request) (*http.Response, error) { + req, err := r.toHTTP() + if err != nil { + return nil, err + } + + // add default headers + req.Header.Set("Content-Type", "application/json") + req.Header.Set("Accept", "application/json") + req.Header.Set("User-Agent", c.useragent()) + + hc := c.httpClient + if hc == nil { + hc = http.DefaultClient + } + + resp, err := hc.Do(req) + return resp, err +} + + +// decodeJsonBody is used to JSON encode a body +func encodeJsonBody(obj interface{}) (io.Reader, error) { + buf := bytes.NewBuffer(nil) + enc := json.NewEncoder(buf) + if err := enc.Encode(obj); err != nil { + return nil, err + } + return buf, nil +} + + +// decodeJsonBody is used to JSON decode a body +func decodeJsonBody(resp *http.Response, out interface{}) error { + dec := json.NewDecoder(resp.Body) + return dec.Decode(out) +} + +// safeRespClose is used to close a respone body +func safeRespClose(resp *http.Response) { + if resp != nil { + if err := resp.Body.Close(); err != nil { + // ignore error since it is closing a response body + } + } +} + +// responseError is used to parse a response into a pulsar error +func responseError(resp *http.Response) error { + var e Error + body, err := ioutil.ReadAll(resp.Body) + if err != nil { + e.Reason = err.Error() + e.Code = resp.StatusCode + return e + } + + jsonErr := json.Unmarshal(body, &e) + + if jsonErr != nil { + e.Code = http.StatusPartialContent + } else { + e.Code = resp.StatusCode + + if e.Reason == "" { + e.Reason = unknownErrorReason + } + } + + return e +} + +// respIsOk is used to validate a successful http status code +func respIsOk(resp *http.Response) bool { + return resp.StatusCode >= http.StatusOK && resp.StatusCode <= http.StatusNoContent +} + +// checkSuccessful checks for a valid response and parses an error +func checkSuccessful(resp *http.Response, err error) (*http.Response, error) { + if err != nil { + safeRespClose(resp) + return nil, err + } + + if !respIsOk(resp) { + defer safeRespClose(resp) + return nil, responseError(resp) + } + + return resp, nil +} + +func endpoint(parts ...string) string { + return path.Join(parts...) +} + diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go new file mode 100644 index 0000000000..dd5f67e825 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -0,0 +1,48 @@ +package pulsar + +// Clusters is used to access the cluster endpoints. + +type Clusters interface { + List() ([]string, error) + Get(string) (ClusterData, error) + Create(ClusterData) error + Delete(string) error +} + +type clusters struct { + client *client + basePath string +} + +func (c *client) Clusters() Clusters { + return &clusters{ + client: c, + basePath:"/clusters", + } +} + +func (c *clusters) List() ([]string, error) { + var clusters []string + err := c.client.get(c.client.endpoint(c.basePath), &clusters) + return clusters, err +} + +func (c *clusters) Get(name string) (ClusterData, error) { + cdata := ClusterData{} + endpoint := c.client.endpoint(c.basePath, name) + err := c.client.get(endpoint, &cdata) + return cdata, err +} + +func (c *clusters) Create(cdata ClusterData) error { + endpoint := c.client.endpoint(c.basePath, cdata.Name) + err := c.client.put(endpoint, &cdata, nil) + return err +} + +func (c *clusters) Delete(name string) error { + endpoint := c.client.endpoint(c.basePath, name) + return c.client.delete(endpoint, nil) +} + + diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go new file mode 100644 index 0000000000..2ad69c717b --- /dev/null +++ b/pulsaradmin/pkg/pulsar/data.go @@ -0,0 +1,11 @@ +package pulsar + +// ClusterData information on a cluster +type ClusterData struct { + Name string `json:"-"` + ServiceURL string `json:"serviceUrl"` + ServiceURLTls string `json:"serviceUrlTls"` + BrokerServiceURL string `json:"brokerServiceUrl"` + BrokerServiceURLTls string `json:"brokerServiceUrlTls"` + PeerClusterNames []string `json:"peerClusterNames"` +} diff --git a/pulsaradmin/pkg/pulsar/errors.go b/pulsaradmin/pkg/pulsar/errors.go new file mode 100644 index 0000000000..19cfd2983c --- /dev/null +++ b/pulsaradmin/pkg/pulsar/errors.go @@ -0,0 +1,19 @@ +package pulsar + +import "fmt" + +const unknownErrorReason = "Unknown pulsar error" + +type Error struct { + Reason string `json:"reason"` + Code int +} + +func (e Error) Error() string { + return fmt.Sprintf("code: %d reason: %s", e.Code, e.Reason) +} + +func IsAdminError(err error) bool { + _, ok := err.(Error) + return ok +} diff --git a/pulsaradmin/pkg/pulsar/utils.go b/pulsaradmin/pkg/pulsar/utils.go new file mode 100644 index 0000000000..fa62b7c2b2 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/utils.go @@ -0,0 +1,7 @@ +package pulsar + +import "fmt" + +func makeHttpPath(apiVersion string, componentPath string) string { + return fmt.Sprintf("/admin/%s%s", apiVersion, componentPath) +} From 9f80a95ac23c92aef80ab253e68a4118addf2e43 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Tue, 27 Aug 2019 21:14:03 +0800 Subject: [PATCH 085/348] Add post request for project (streamnative/pulsarctl#7) Signed-off-by: xiaolong.ran Add post request for project --- pulsaradmin/pkg/pulsar/admin.go | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index a6f5768dd0..9053eac5fb 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -128,6 +128,28 @@ func (c *client) delete(endpoint string, obj interface{}) error { return nil } +func (c *client) post(endpoint string, in, obj interface{}) error { + req, err := c.newRequest(http.MethodPost, endpoint) + if err != nil { + return err + } + req.obj = in + + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + + if obj != nil { + if err := decodeJsonBody(resp, &obj); err != nil { + return err + } + } + + return nil +} + type request struct { method string url *url.URL From 64b9813c0d91d2aeca381461e1ea254491fba84a Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 28 Aug 2019 16:18:25 +0800 Subject: [PATCH 086/348] Add cluster commands `get` (streamnative/pulsarctl#4) Add cluster commands `get` --- pulsaradmin/go.mod | 4 ++ pulsaradmin/pkg/pulsar/descriptions.go | 62 +++++++++++++++++++++ pulsaradmin/pkg/pulsar/descriptions_test.go | 52 +++++++++++++++++ 3 files changed, 118 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/descriptions.go create mode 100644 pulsaradmin/pkg/pulsar/descriptions_test.go diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index 7f2ca09017..48bd004b82 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -4,11 +4,15 @@ go 1.12 require ( github.com/fatih/color v1.7.0 // indirect + github.com/google/go-cmp v0.3.1 // indirect github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b // indirect github.com/mattn/go-colorable v0.1.2 // indirect github.com/mattn/go-runewidth v0.0.4 // indirect github.com/olekukonko/tablewriter v0.0.1 + github.com/pkg/errors v0.8.1 // indirect github.com/spf13/cobra v0.0.5 github.com/spf13/pflag v1.0.3 + github.com/stretchr/objx v0.2.0 // indirect + github.com/stretchr/testify v1.3.0 ) diff --git a/pulsaradmin/pkg/pulsar/descriptions.go b/pulsaradmin/pkg/pulsar/descriptions.go new file mode 100644 index 0000000000..22094f786e --- /dev/null +++ b/pulsaradmin/pkg/pulsar/descriptions.go @@ -0,0 +1,62 @@ +package pulsar + +import "strings" + +var SPACES = " " +var USED_FOR = "USED FOR:" +var PERMISSION = "REQUIRED PERMISSION:" +var EXAMPLES = "EXAMPLES:" +var OUTPUT = "OUTPUT:" + +type LongDescription struct { + CommandUsedFor string + CommandPermission string + CommandExamples []Example + CommandOutput []Output +} + +type Example struct { + Desc string + Command string +} + +type Output struct { + Desc string + Out string +} + +func (desc *LongDescription) ToString() string { + return USED_FOR + "\n" + + SPACES + desc.CommandUsedFor + "\n\n" + + PERMISSION + "\n" + + SPACES + desc.CommandPermission + "\n\n" + + EXAMPLES + "\n" + + desc.exampleToString() + "\n" + + OUTPUT + "\n" + + desc.outputToString() +} + +func (desc *LongDescription) exampleToString() string { + var result string + for _, v := range desc.CommandExamples { + result += SPACES + "#" + v.Desc + "\n" + SPACES + v.Command + "\n" + } + return result +} + +func (desc *LongDescription) outputToString() string { + var result string + for _, v := range desc.CommandOutput { + result += SPACES + "#" + v.Desc + "\n" + makeSpace(v.Out) + "\n" + } + return result +} + +func makeSpace(s string) string { + var res string + lines := strings.Split(s, "\n") + for _, l := range lines { + res += SPACES + l + "\n" + } + return res +} diff --git a/pulsaradmin/pkg/pulsar/descriptions_test.go b/pulsaradmin/pkg/pulsar/descriptions_test.go new file mode 100644 index 0000000000..99a5abaf02 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/descriptions_test.go @@ -0,0 +1,52 @@ +package pulsar + +import ( + "github.com/stretchr/testify/assert" + "testing" +) + +func TestLongDescription_exampleToString(t *testing.T) { + desc := LongDescription{} + example := Example{ + Desc: "command description", + Command: "command", + } + desc.CommandExamples = []Example{example} + res := desc.exampleToString() + + expect := " #command description\n" + + " command\n" + + assert.Equal(t, expect, res) +} + +func TestLongDescription_ToString(t *testing.T) { + desc := LongDescription{} + desc.CommandUsedFor = "command used for" + desc.CommandPermission = "command permission" + example := Example{} + example.Desc = "command description" + example.Command = "command" + desc.CommandExamples = []Example{example} + out := Output{ + Desc: "Output", + Out: "Out line 1\nOut line 2", + } + desc.CommandOutput = []Output{out} + + expect := "USED FOR:\n" + + " " + desc.CommandUsedFor + "\n\n" + + "REQUIRED PERMISSION:\n" + + " " + desc.CommandPermission + "\n\n" + + "EXAMPLES:\n" + + " " + "#" + example.Desc + "\n" + + " " + example.Command + "\n\n" + + "OUTPUT:\n" + + " " + "#" + out.Desc + "\n" + + " " + "Out line 1" + "\n" + + " " + "Out line 2" + "\n\n" + + result := desc.ToString() + + assert.Equal(t, expect, result) +} From 54e7ebbc0706f1733a50322fa684d46c41d14edd Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 29 Aug 2019 12:11:03 +0800 Subject: [PATCH 087/348] Make API version configurable (streamnative/pulsarctl#16) *Motivation* Pulsar admin requests will use different API version to request broker. So we need to make the API version configurable. *Modifications* - Provide an API version to use. - Send requests with different API version settings. --- pulsaradmin/pkg/pulsar/admin.go | 21 ++++++++----------- pulsaradmin/pkg/pulsar/api_version.go | 24 ++++++++++++++++++++++ pulsaradmin/pkg/pulsar/api_version_test.go | 12 +++++++++++ 3 files changed, 45 insertions(+), 12 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/api_version.go create mode 100644 pulsaradmin/pkg/pulsar/api_version_test.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 9053eac5fb..7941e1a397 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -12,13 +12,14 @@ import ( ) const ( - DefaultWebServiceURL = "http://localhost:8080" + DefaultWebServiceURL = "http://localhost:8080" ) // Config is used to configure the admin client type Config struct { WebServiceUrl string HttpClient *http.Client + ApiVersion ApiVersion } // DefaultConfig returns a default configuration for the pulsar admin client @@ -51,7 +52,7 @@ func New(config *Config) Client { c := &client{ // TODO: make api version configurable - apiVersion: "v2", + apiVersion: config.ApiVersion.String(), webServiceUrl: config.WebServiceUrl, } @@ -152,10 +153,10 @@ func (c *client) post(endpoint string, in, obj interface{}) error { type request struct { method string - url *url.URL + url *url.URL params url.Values - obj interface{} + obj interface{} body io.Reader } @@ -182,7 +183,6 @@ func (r *request) toHTTP() (*http.Request, error) { return req, nil } - func (c *client) newRequest(method, path string) (*request, error) { base, _ := url.Parse(c.webServiceUrl) u, err := url.Parse(path) @@ -193,9 +193,9 @@ func (c *client) newRequest(method, path string) (*request, error) { method: method, url: &url.URL{ Scheme: base.Scheme, - User: base.User, - Host: base.Host, - Path: endpoint(base.Path, u.Path), + User: base.User, + Host: base.Host, + Path: endpoint(base.Path, u.Path), }, params: make(url.Values), } @@ -227,7 +227,6 @@ func (c *client) doRequest(r *request) (*http.Response, error) { return resp, err } - // decodeJsonBody is used to JSON encode a body func encodeJsonBody(obj interface{}) (io.Reader, error) { buf := bytes.NewBuffer(nil) @@ -238,7 +237,6 @@ func encodeJsonBody(obj interface{}) (io.Reader, error) { return buf, nil } - // decodeJsonBody is used to JSON decode a body func decodeJsonBody(resp *http.Response, out interface{}) error { dec := json.NewDecoder(resp.Body) @@ -255,7 +253,7 @@ func safeRespClose(resp *http.Response) { } // responseError is used to parse a response into a pulsar error -func responseError(resp *http.Response) error { +func responseError(resp *http.Response) error { var e Error body, err := ioutil.ReadAll(resp.Body) if err != nil { @@ -302,4 +300,3 @@ func checkSuccessful(resp *http.Response, err error) (*http.Response, error) { func endpoint(parts ...string) string { return path.Join(parts...) } - diff --git a/pulsaradmin/pkg/pulsar/api_version.go b/pulsaradmin/pkg/pulsar/api_version.go new file mode 100644 index 0000000000..ba055ef191 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/api_version.go @@ -0,0 +1,24 @@ +package pulsar + +type ApiVersion int +const ( + V1 ApiVersion = iota + V2 + V3 +) + +const DefaultApiVersion = "v2" + +func (v ApiVersion) String() string { + switch v { + case V1: + return "v1" + case V2: + return "v2" + case V3: + return "v3" + } + + return DefaultApiVersion +} + diff --git a/pulsaradmin/pkg/pulsar/api_version_test.go b/pulsaradmin/pkg/pulsar/api_version_test.go new file mode 100644 index 0000000000..d9982661ac --- /dev/null +++ b/pulsaradmin/pkg/pulsar/api_version_test.go @@ -0,0 +1,12 @@ +package pulsar + +import ( + "github.com/stretchr/testify/assert" + "testing" +) + +func TestApiVersion_String(t *testing.T) { + assert.Equal(t, "v1", V1.String()) + assert.Equal(t, "v2", V2.String()) + assert.Equal(t, "v3", V3.String()) +} \ No newline at end of file From ab9e0195d6461077fef809851587850311f60ae0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Fri, 30 Aug 2019 16:21:49 +0800 Subject: [PATCH 088/348] Add auth logic for `pulsarctl` (streamnative/pulsarctl#20) Master Issue: streamnative/pulsarctl#2 Add auth logic for `pulsarctl` --- pulsaradmin/pkg/auth/tls.go | 56 ++++++++++++++++++++ pulsaradmin/pkg/pulsar/admin.go | 90 ++++++++++++++++++++++++++++++--- pulsaradmin/pkg/pulsar/utils.go | 4 +- 3 files changed, 143 insertions(+), 7 deletions(-) create mode 100644 pulsaradmin/pkg/auth/tls.go diff --git a/pulsaradmin/pkg/auth/tls.go b/pulsaradmin/pkg/auth/tls.go new file mode 100644 index 0000000000..faeead661b --- /dev/null +++ b/pulsaradmin/pkg/auth/tls.go @@ -0,0 +1,56 @@ +// 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 auth + +import "crypto/tls" + +type TlsAuthProvider struct { + certificatePath string + privateKeyPath string +} + +// NewAuthenticationTLSWithParams initialize the authentication provider with map param. +func NewAuthenticationTLSWithParams(params map[string]string) *TlsAuthProvider { + return NewAuthenticationTLS( + params["tlsCertFile"], + params["tlsKeyFile"], + ) +} + +// NewAuthenticationTLS initialize the authentication provider +func NewAuthenticationTLS(certificatePath string, privateKeyPath string) *TlsAuthProvider { + return &TlsAuthProvider{ + certificatePath: certificatePath, + privateKeyPath: privateKeyPath, + } +} + +func (p *TlsAuthProvider) Init() error { + // Try to read certificates immediately to provide better error at startup + _, err := p.GetTLSCertificate() + return err +} + +func (p *TlsAuthProvider) Name() string { + return "tls" +} + +func (p *TlsAuthProvider) GetTLSCertificate() (*tls.Certificate, error) { + cert, err := tls.LoadX509KeyPair(p.certificatePath, p.privateKeyPath) + return &cert, err +} diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 7941e1a397..5116af4b3a 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -2,13 +2,19 @@ package pulsar import ( "bytes" + "crypto/tls" + "crypto/x509" "encoding/json" + "errors" "fmt" "io" "io/ioutil" "net/http" "net/url" "path" + "strings" + + "github.com/streamnative/pulsar-admin-go/pkg/auth" ) const ( @@ -20,6 +26,15 @@ type Config struct { WebServiceUrl string HttpClient *http.Client ApiVersion ApiVersion + + Auth *auth.TlsAuthProvider + AuthParams string + TlsOptions *TLSOptions +} + +type TLSOptions struct { + TrustCertsFilePath string + AllowInsecureConnection bool } // DefaultConfig returns a default configuration for the pulsar admin client @@ -27,6 +42,10 @@ func DefaultConfig() *Config { config := &Config{ WebServiceUrl: DefaultWebServiceURL, HttpClient: http.DefaultClient, + + TlsOptions: &TLSOptions{ + AllowInsecureConnection: false, + }, } return config } @@ -40,23 +59,78 @@ type client struct { webServiceUrl string apiVersion string httpClient *http.Client + + // TLS config + auth *auth.TlsAuthProvider + authParams string + tlsOptions *TLSOptions + transport *http.Transport } // New returns a new client -func New(config *Config) Client { - defConfig := DefaultConfig() - +func New(config *Config) (Client, error) { if len(config.WebServiceUrl) == 0 { - config.WebServiceUrl = defConfig.WebServiceUrl + config.WebServiceUrl = DefaultWebServiceURL } c := &client{ - // TODO: make api version configurable apiVersion: config.ApiVersion.String(), webServiceUrl: config.WebServiceUrl, } - return c + if strings.HasPrefix(c.webServiceUrl, "https://") { + c.authParams = config.AuthParams + c.tlsOptions = config.TlsOptions + mapAuthParams := make(map[string]string) + + err := json.Unmarshal([]byte(c.authParams), &mapAuthParams) + if err != nil { + return nil, err + } + c.auth = auth.NewAuthenticationTLSWithParams(mapAuthParams) + + tlsConf, err := c.getTLSConfig() + if err != nil { + return nil, err + } + + c.transport = &http.Transport{ + MaxIdleConnsPerHost: 10, + TLSClientConfig: tlsConf, + } + } + + return c, nil +} + +func (c *client) getTLSConfig() (*tls.Config, error) { + tlsConfig := &tls.Config{ + InsecureSkipVerify: c.tlsOptions.AllowInsecureConnection, + } + + if c.tlsOptions.TrustCertsFilePath != "" { + caCerts, err := ioutil.ReadFile(c.tlsOptions.TrustCertsFilePath) + if err != nil { + return nil, err + } + + tlsConfig.RootCAs = x509.NewCertPool() + ok := tlsConfig.RootCAs.AppendCertsFromPEM(caCerts) + if !ok { + return nil, errors.New("failed to parse root CAs certificates") + } + } + + cert, err := c.auth.GetTLSCertificate() + if err != nil { + return nil, err + } + + if cert != nil { + tlsConfig.Certificates = []tls.Certificate{*cert} + } + + return tlsConfig, nil } func (c *client) endpoint(componentPath string, parts ...string) string { @@ -189,6 +263,7 @@ func (c *client) newRequest(method, path string) (*request, error) { if err != nil { return nil, err } + req := &request{ method: method, url: &url.URL{ @@ -222,6 +297,9 @@ func (c *client) doRequest(r *request) (*http.Response, error) { if hc == nil { hc = http.DefaultClient } + if c.transport != nil { + hc.Transport = c.transport + } resp, err := hc.Do(req) return resp, err diff --git a/pulsaradmin/pkg/pulsar/utils.go b/pulsaradmin/pkg/pulsar/utils.go index fa62b7c2b2..740ed4b3ab 100644 --- a/pulsaradmin/pkg/pulsar/utils.go +++ b/pulsaradmin/pkg/pulsar/utils.go @@ -1,6 +1,8 @@ package pulsar -import "fmt" +import ( + "fmt" +) func makeHttpPath(apiVersion string, componentPath string) string { return fmt.Sprintf("/admin/%s%s", apiVersion, componentPath) From 25c6db28995143a3f4d79ff1bc8eab7b2637fbce Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 2 Sep 2019 12:11:41 +0800 Subject: [PATCH 089/348] Add command cluster `update` (streamnative/pulsarctl#17) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Master issue: streamnative/pulsarctl#2 ``` ➜ pulsarctl git:(cluster_update) ✗ ./pulsarctl clusters update -h USED FOR: This command is used for updating the cluster data of the specified cluster. REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #updating the web service url of the pulsarctl clusters update --url http://example:8080 #updating the tls secured web service url of the pulsarctl clusters update --url-tls https://example:8080 #updating the broker service url of the pulsarctl clusters update --broker-url pulsar://example:6650 #updating the tls secured web service url of the pulsarctl clusters update --broker-url-tls pulsar+ssl://example:6650 #registered as a peer-cluster of the clusters pulsarctl clusters update -p , OUTPUT: #normal output Cluster updated #output of doesn't specified a cluster name [✖] only one argument is allowed to be used as a name #Output of cluster doesn't exist [✖] code: 404 reason: Cluster does not exist Usage: pulsarctl clusters update [flags] Aliases: update, update ClusterData flags: --url string Pulsar cluster web service url, e.g. http://example.pulsar.io:8080 --url-tls string Pulsar cluster tls secured web service url, e.g. https://example.pulsar.io:8443 --broker-url string Pulsar cluster broker service url, e.g. pulsar://example.pulsar.io:6650 --broker-url-tls string Pulsar cluster tls secured broker service url, e.g. pulsar+ssl://example.pulsar.io:6651 -p, --peer-cluster stringArray Cluster to be registered as a peer-cluster of this cluster. Common flags: -s, --admin-service-url string The admin web service url that pulsarctl connects to. (default "http://localhost:8080") -C, --color string toggle colorized logs (true,false,fabulous) (default "true") -h, --help help for this command -v, --verbose int set log level, use 0 to silence, 4 for debugging (default 3) Use 'pulsarctl clusters update [command] --help' for more information about a command. ``` --- pulsaradmin/pkg/pulsar/cluster.go | 14 +++++++++----- pulsaradmin/pkg/pulsar/descriptions.go | 4 ++-- pulsaradmin/pkg/pulsar/descriptions_test.go | 2 +- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index dd5f67e825..4c4acd690e 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -7,17 +7,18 @@ type Clusters interface { Get(string) (ClusterData, error) Create(ClusterData) error Delete(string) error + Update(ClusterData) error } type clusters struct { - client *client - basePath string + client *client + basePath string } func (c *client) Clusters() Clusters { return &clusters{ - client: c, - basePath:"/clusters", + client: c, + basePath: "/clusters", } } @@ -45,4 +46,7 @@ func (c *clusters) Delete(name string) error { return c.client.delete(endpoint, nil) } - +func (c *clusters) Update(cdata ClusterData) error { + endpoint := c.client.endpoint(c.basePath, cdata.Name) + return c.client.post(endpoint, &cdata, nil) +} diff --git a/pulsaradmin/pkg/pulsar/descriptions.go b/pulsaradmin/pkg/pulsar/descriptions.go index 22094f786e..8cb29804a8 100644 --- a/pulsaradmin/pkg/pulsar/descriptions.go +++ b/pulsaradmin/pkg/pulsar/descriptions.go @@ -31,7 +31,7 @@ func (desc *LongDescription) ToString() string { PERMISSION + "\n" + SPACES + desc.CommandPermission + "\n\n" + EXAMPLES + "\n" + - desc.exampleToString() + "\n" + + desc.exampleToString() + OUTPUT + "\n" + desc.outputToString() } @@ -39,7 +39,7 @@ func (desc *LongDescription) ToString() string { func (desc *LongDescription) exampleToString() string { var result string for _, v := range desc.CommandExamples { - result += SPACES + "#" + v.Desc + "\n" + SPACES + v.Command + "\n" + result += SPACES + "#" + v.Desc + "\n" + SPACES + v.Command + "\n\n" } return result } diff --git a/pulsaradmin/pkg/pulsar/descriptions_test.go b/pulsaradmin/pkg/pulsar/descriptions_test.go index 99a5abaf02..fa643185bc 100644 --- a/pulsaradmin/pkg/pulsar/descriptions_test.go +++ b/pulsaradmin/pkg/pulsar/descriptions_test.go @@ -15,7 +15,7 @@ func TestLongDescription_exampleToString(t *testing.T) { res := desc.exampleToString() expect := " #command description\n" + - " command\n" + " command\n\n" assert.Equal(t, expect, res) } From 30be14badfca25a1beff297f664bcc792fed5dce Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 3 Sep 2019 02:38:13 +0800 Subject: [PATCH 090/348] Add command cluster `update-peer-clusters` (streamnative/pulsarctl#18) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Master issue: streamnative/pulsarctl#2 output : ``` USED FOR: This command is used for updating peer clusters. REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #updating the peer clusters pulsarctl clusters update-peer-clusters -p cluster-a,cluster-b OUTPUT: #normal output peer clusters updated #output of doesn't specified a cluster name [✖] only one argument is allowed to be used as a name #output of cluster doesn't exist [✖] code: 404 reason: Cluster does not exist Usage: pulsarctl clusters update-peer-clusters [flags] Aliases: update-peer-clusters, upc Update peer clusters flags: -p, --peer-cluster stringArray Cluster to be registered as a peer-cluster of this cluster Common flags: -s, --admin-service-url string The admin web service url that pulsarctl connects to. (default "http://localhost:8080") -C, --color string toggle colorized logs (true,false,fabulous) (default "true") -h, --help help for this command -v, --verbose int set log level, use 0 to silence, 4 for debugging (default 3) Use 'pulsarctl clusters update-peer-clusters [command] --help' for more information about a command. ``` --- pulsaradmin/pkg/pulsar/cluster.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index 4c4acd690e..c9517d4f78 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -8,6 +8,7 @@ type Clusters interface { Create(ClusterData) error Delete(string) error Update(ClusterData) error + UpdatePeerClusters(string, []string) error } type clusters struct { @@ -50,3 +51,8 @@ func (c *clusters) Update(cdata ClusterData) error { endpoint := c.client.endpoint(c.basePath, cdata.Name) return c.client.post(endpoint, &cdata, nil) } + +func (c *clusters) UpdatePeerClusters(cluster string, peerClusters []string) error { + endpoint := c.client.endpoint(c.basePath, cluster, "peers") + return c.client.post(endpoint, peerClusters, nil) +} From 61615ea05068d3f9755302f1b06834a2669246c2 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 3 Sep 2019 10:35:19 +0800 Subject: [PATCH 091/348] Add command cluster `get-peer-clusters` (streamnative/pulsarctl#19) * Add command cluster `get-peer-clusters` --- Master Issue: streamnative/pulsarctl#2 * Fix long description --- pulsaradmin/pkg/pulsar/cluster.go | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index c9517d4f78..0902a89ad3 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -9,6 +9,7 @@ type Clusters interface { Delete(string) error Update(ClusterData) error UpdatePeerClusters(string, []string) error + GetPeerClusters(string) ([]string, error) } type clusters struct { @@ -53,6 +54,12 @@ func (c *clusters) Update(cdata ClusterData) error { } func (c *clusters) UpdatePeerClusters(cluster string, peerClusters []string) error { - endpoint := c.client.endpoint(c.basePath, cluster, "peers") + endpoint := c.client.endpoint(c.basePath, cluster, "peers") return c.client.post(endpoint, peerClusters, nil) } +func (c *clusters) GetPeerClusters(name string) ([]string, error) { + var peerClusters []string + endpoint := c.client.endpoint(c.basePath, name, "peers") + err := c.client.get(endpoint, &peerClusters) + return peerClusters, err +} From 1f646f5aa848e72636c568c5140ba3e5d31f0edb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Tue, 3 Sep 2019 11:50:43 +0800 Subject: [PATCH 092/348] Add create cmd of Pulsar Functions (streamnative/pulsarctl#15) Add create cmd of Pulsar Functions --- pulsaradmin/go.mod | 2 + pulsaradmin/pkg/pulsar/WindowConfing.go | 39 +++++ pulsaradmin/pkg/pulsar/admin.go | 43 ++++-- pulsaradmin/pkg/pulsar/cluster.go | 3 +- pulsaradmin/pkg/pulsar/consumerConfig.go | 25 ++++ pulsaradmin/pkg/pulsar/data.go | 52 ++++++- pulsaradmin/pkg/pulsar/functionConfg.go | 87 +++++++++++ pulsaradmin/pkg/pulsar/functions.go | 176 +++++++++++++++++++++++ pulsaradmin/pkg/pulsar/resources.go | 56 ++++++++ pulsaradmin/pkg/pulsar/sinkConfig.go | 51 +++++++ pulsaradmin/pkg/pulsar/sourceConfig.go | 45 ++++++ 11 files changed, 563 insertions(+), 16 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/WindowConfing.go create mode 100644 pulsaradmin/pkg/pulsar/consumerConfig.go create mode 100644 pulsaradmin/pkg/pulsar/functionConfg.go create mode 100644 pulsaradmin/pkg/pulsar/functions.go create mode 100644 pulsaradmin/pkg/pulsar/resources.go create mode 100644 pulsaradmin/pkg/pulsar/sinkConfig.go create mode 100644 pulsaradmin/pkg/pulsar/sourceConfig.go diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index 48bd004b82..d2b82cd1ed 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -3,6 +3,7 @@ module github.com/streamnative/pulsar-admin-go go 1.12 require ( + github.com/davecgh/go-spew v1.1.1 github.com/fatih/color v1.7.0 // indirect github.com/google/go-cmp v0.3.1 // indirect github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 @@ -15,4 +16,5 @@ require ( github.com/spf13/pflag v1.0.3 github.com/stretchr/objx v0.2.0 // indirect github.com/stretchr/testify v1.3.0 + gopkg.in/yaml.v2 v2.2.2 ) diff --git a/pulsaradmin/pkg/pulsar/WindowConfing.go b/pulsaradmin/pkg/pulsar/WindowConfing.go new file mode 100644 index 0000000000..cf96ff0c97 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/WindowConfing.go @@ -0,0 +1,39 @@ +// 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 pulsar + +const WindowConfigKey = "__WINDOWCONFIGS__" + +type WindowConfig struct { + WindowLengthCount int + WindowLengthDurationMs int64 + SlidingIntervalCount int + SlidingIntervalDurationMs int64 + LateDataTopic string + MaxLagMs int64 + WatermarkEmitIntervalMs int64 + TimestampExtractorClassName string + ActualWindowFunctionClassName string +} + +func NewDefaultWindowConfing() *WindowConfig { + windowConfig := &WindowConfig{} + + return windowConfig +} + diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 5116af4b3a..53ca6bbd49 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -53,6 +53,7 @@ func DefaultConfig() *Config { // Client provides a client to the Pulsar Restful API type Client interface { Clusters() Clusters + Functions() Functions } type client struct { @@ -209,6 +210,27 @@ func (c *client) post(endpoint string, in, obj interface{}) error { return err } req.obj = in + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + if obj != nil { + if err := decodeJsonBody(resp, &obj); err != nil { + return err + } + } + return nil +} + +func (c *client) postWithMultiPart(endpoint string, in, obj interface{}, body io.Reader, contentType string) error { + req, err := c.newRequest(http.MethodPost, endpoint) + if err != nil { + return err + } + req.obj = in + req.body = body + req.contentType = contentType resp, err := checkSuccessful(c.doRequest(req)) if err != nil { @@ -226,9 +248,10 @@ func (c *client) post(endpoint string, in, obj interface{}) error { } type request struct { - method string - url *url.URL - params url.Values + method string + contentType string + url *url.URL + params url.Values obj interface{} body io.Reader @@ -288,9 +311,14 @@ func (c *client) doRequest(r *request) (*http.Response, error) { return nil, err } - // add default headers - req.Header.Set("Content-Type", "application/json") - req.Header.Set("Accept", "application/json") + if r.contentType != "" { + req.Header.Set("Content-Type", r.contentType) + } else { + // add default headers + req.Header.Set("Content-Type", "application/json") + req.Header.Set("Accept", "application/json") + } + req.Header.Set("User-Agent", c.useragent()) hc := c.httpClient @@ -301,8 +329,7 @@ func (c *client) doRequest(r *request) (*http.Response, error) { hc.Transport = c.transport } - resp, err := hc.Do(req) - return resp, err + return hc.Do(req) } // decodeJsonBody is used to JSON encode a body diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index 0902a89ad3..9465a9e917 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -39,8 +39,7 @@ func (c *clusters) Get(name string) (ClusterData, error) { func (c *clusters) Create(cdata ClusterData) error { endpoint := c.client.endpoint(c.basePath, cdata.Name) - err := c.client.put(endpoint, &cdata, nil) - return err + return c.client.put(endpoint, &cdata, nil) } func (c *clusters) Delete(name string) error { diff --git a/pulsaradmin/pkg/pulsar/consumerConfig.go b/pulsaradmin/pkg/pulsar/consumerConfig.go new file mode 100644 index 0000000000..6d334aff46 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/consumerConfig.go @@ -0,0 +1,25 @@ +// 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 pulsar + +type ConsumerConfig struct { + SchemaType string + SerdeClassName string + IsRegexPattern bool + ReceiverQueueSize int +} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index 2ad69c717b..dbad0508d3 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -2,10 +2,50 @@ package pulsar // ClusterData information on a cluster type ClusterData struct { - Name string `json:"-"` - ServiceURL string `json:"serviceUrl"` - ServiceURLTls string `json:"serviceUrlTls"` - BrokerServiceURL string `json:"brokerServiceUrl"` - BrokerServiceURLTls string `json:"brokerServiceUrlTls"` - PeerClusterNames []string `json:"peerClusterNames"` + Name string `json:"-"` + ServiceURL string `json:"serviceUrl"` + ServiceURLTls string `json:"serviceUrlTls"` + BrokerServiceURL string `json:"brokerServiceUrl"` + BrokerServiceURLTls string `json:"brokerServiceUrlTls"` + PeerClusterNames []string `json:"peerClusterNames"` +} + +// FunctionData information for a Pulsar Function +type FunctionData struct { + FQFN string `json:"fqfn"` + Tenant string `json:"tenant"` + Namespace string `json:"namespace"` + FuncName string `json:"functionName"` + ClassName string `json:"className"` + Jar string `json:"jarFile"` + Py string `json:"pyFile"` + Go string `json:"goFile"` + Inputs string `json:"inputs"` + TopicsPattern string `json:"topicsPattern"` + Output string `json:"output"` + LogTopic string `json:"logTopic"` + SchemaType string `json:"schemaType"` + CustomSerDeInputs string `json:"customSerdeInputString"` + CustomSchemaInput string `json:"customSchemaInputString"` + OutputSerDeClassName string `json:"outputSerdeClassName"` + FunctionConfigFile string `json:"fnConfigFile"` + ProcessingGuarantees string `json:"processingGuarantees"` + UserConfig string `json:"userConfigString"` + RetainOrdering bool `json:"retainOrdering"` + SubsName string `json:"subsName"` + Parallelism int `json:"parallelism"` + CPU float64 `json:"cpu"` + RAM int64 `json:"ram"` + Disk int64 `json:"disk"` + WindowLengthCount int `json:"windowLengthCount"` + WindowLengthDurationMs int64 `json:"windowLengthDurationMs"` + SlidingIntervalCount int `json:"slidingIntervalCount"` + SlidingIntervalDurationMs int64 `json:"slidingIntervalDurationMs"` + AutoAck bool `json:"autoAck"` + TimeoutMs int64 `json:"timeoutMs"` + MaxMessageRetries int `json:"maxMessageRetries"` + DeadLetterTopic string `json:"deadLetterTopic"` + + FuncConf *FunctionConfig `json:"-"` + UserCodeFile string `json:"-"` } diff --git a/pulsaradmin/pkg/pulsar/functionConfg.go b/pulsaradmin/pkg/pulsar/functionConfg.go new file mode 100644 index 0000000000..01c0ffdfbf --- /dev/null +++ b/pulsaradmin/pkg/pulsar/functionConfg.go @@ -0,0 +1,87 @@ +// 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 pulsar + +type ProcessingGuarantees int + +type Runtime int + +const ( + AtLeasetOnce ProcessingGuarantees = iota + AtMostOnce + EffectivelyOnce +) + +const ( + Java Runtime = iota + Python + Go +) + +type FunctionConfig struct { + // Any flags that you want to pass to the runtime. + // note that in thread mode, these flags will have no impact + RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` + + Tenant string `json:"tenant" yaml:"tenant"` + Namespace string `json:"namespace" yaml:"namespace"` + Name string `json:"name" yaml:"name"` + ClassName string `json:"className" yaml:"className"` + + Inputs []string `json:"inputs" yaml:"inputs"` + CustomSerdeInputs map[string]string `json:"customSerdeInputs" yaml:"customSerdeInputs"` + TopicsPattern *string `json:"topicsPattern" yaml:"topicsPattern"` + CustomSchemaInputs map[string]string `json:"customSchemaInputs" yaml:"customSchemaInputs"` + + // A generalized way of specifying inputs + InputSpecs map[string]ConsumerConfig `json:"inputSpecs" yaml:"inputSpecs"` + + Output string `json:"output" yaml:"output"` + + // Represents either a builtin schema type (eg: 'avro', 'json', ect) or the class name for a Schema implementation + OutputSchemaType string `json:"outputSchemaType" yaml:"outputSchemaType"` + + OutputSerdeClassName string `json:"outputSerdeClassName" yaml:"outputSerdeClassName"` + LogTopic string `json:"logTopic" yaml:"logTopic"` + ProcessingGuarantees ProcessingGuarantees `json:"processingGuarantees" yaml:"processingGuarantees"` + RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` + UserConfig map[string]interface{} `json:"userConfig" yaml:"userConfig"` + + // This is a map of secretName(aka how the secret is going to be + // accessed in the function via context) to an object that + // encapsulates how the secret is fetched by the underlying + // secrets provider. The type of an value here can be found by the + // SecretProviderConfigurator.getSecretObjectType() method. + Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` + + Runtime Runtime `json:"runtime" yaml:"runtime"` + AutoAck bool `json:"autoAck" yaml:"autoAck"` + MaxMessageRetries int `json:"maxMessageRetries" yaml:"maxMessageRetries"` + DeadLetterTopic string `json:"deadLetterTopic" yaml:"deadLetterTopic"` + SubName string `json:"subName" yaml:"subName"` + Parallelism int `json:"parallelism" yaml:"parallelism"` + Resources *Resources `json:"resources" yaml:"resources"` + FQFN string `json:"fqfn" yaml:"fqfn"` + WindowConfig *WindowConfig `json:"windowConfig" yaml:"windowConfig"` + TimeoutMs *int64 `json:"timeoutMs" yaml:"timeoutMs"` + Jar string `json:"jar" yaml:"jar"` + Py string `json:"py" yaml:"py"` + Go string `json:"go" yaml:"go"` + // Whether the subscriptions the functions created/used should be deleted when the functions is deleted + CleanupSubscription bool `json:"cleanupSubscription" yaml:"cleanupSubscription"` +} diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go new file mode 100644 index 0000000000..db09a619b6 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -0,0 +1,176 @@ +// 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 pulsar + +import ( + "bytes" + "encoding/json" + "fmt" + "io" + "mime/multipart" + "net/textproto" + "os" + "path/filepath" + "strings" +) + +type Functions interface { + // Create a new function. + CreateFunc(data *FunctionConfig, fileName string) error + + // Create a new function by providing url from which fun-pkg can be downloaded. supported url: http/file + // eg: + // File: file:/dir/fileName.jar + // Http: http://www.repo.com/fileName.jar + // + // @param functionConfig + // the function configuration object + // @param pkgUrl + // url from which pkg can be downloaded + CreateFuncWithUrl(data *FunctionConfig, pkgUrl string) error +} + +type functions struct { + client *client + basePath string +} + +func (c *client) Functions() Functions { + return &functions{ + client: c, + basePath: "/functions", + } +} + +func (f *functions) createStringFromField(w *multipart.Writer, value string) (io.Writer, error) { + h := make(textproto.MIMEHeader) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, "functionConfig")) + h.Set("Content-Type", "application/json") + return w.CreatePart(h) +} + +func (f *functions) createTextFromFiled(w *multipart.Writer, value string) (io.Writer, error) { + h := make(textproto.MIMEHeader) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, "url")) + h.Set("Content-Type", "text/plain") + return w.CreatePart(h) +} + +func (f *functions) CreateFunc(funcConf *FunctionConfig, fileName string) error { + endpoint := f.client.endpoint(f.basePath, funcConf.Tenant, funcConf.Namespace, funcConf.Name) + + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + jsonData, err := json.Marshal(funcConf) + if err != nil { + return err + } + + stringWriter, err := f.createStringFromField(multiPartWriter, "functionConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { + // If the function code is built in, we don't need to submit here + file, err := os.Open(fileName) + if err != nil { + return err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) + + if err != nil { + return err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = f.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} + +func (f *functions) CreateFuncWithUrl(funcConf *FunctionConfig, pkgUrl string) error { + endpoint := f.client.endpoint(f.basePath, funcConf.Tenant, funcConf.Namespace, funcConf.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + textWriter, err := f.createTextFromFiled(multiPartWriter, "url") + if err != nil { + return err + } + + _, err = textWriter.Write([]byte(pkgUrl)) + if err != nil { + return err + } + + jsonData, err := json.Marshal(funcConf) + if err != nil { + return err + } + + stringWriter, err := f.createStringFromField(multiPartWriter, "functionConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = f.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} diff --git a/pulsaradmin/pkg/pulsar/resources.go b/pulsaradmin/pkg/pulsar/resources.go new file mode 100644 index 0000000000..58730aad99 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/resources.go @@ -0,0 +1,56 @@ +// 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 pulsar + +type Resources struct { + CPU float64 + Disk int64 + Ram int64 +} + +func NewDefaultResources() *Resources { + resources:= &Resources{ + //Default cpu is 1 core + CPU: 1, + // Default memory is 1GB + Disk: 1073741824, + // Default disk is 10GB + Ram: 10737418240, + } + + return resources +} + +func NewResources(resources *Resources) *Resources { + if resources == nil { + return NewDefaultResources() + } + + if resources.CPU == 0 { + resources.CPU = 1 + } + + if resources.Disk == 0 { + resources.Disk = 1073741824 + } + + if resources.Ram == 0 { + resources.Ram = 10737418240 + } + return resources +} diff --git a/pulsaradmin/pkg/pulsar/sinkConfig.go b/pulsaradmin/pkg/pulsar/sinkConfig.go new file mode 100644 index 0000000000..75dc3cd082 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/sinkConfig.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 pulsar + +type SinkConfig struct { + Tenant string + Namespace string + Name string + ClassName string + + SourceSubscriptionName string + Inputs []string + TopicToSerdeClassName map[string]string + TopicsPattern string + TopicToSchemaType map[string]string + InputSpecs map[string]ConsumerConfig + Configs map[string]interface{} + + // This is a map of secretName(aka how the secret is going to be + // accessed in the function via context) to an object that + // encapsulates how the secret is fetched by the underlying + // secrets provider. The type of an value here can be found by the + // SecretProviderConfigurator.getSecretObjectType() method. + Secrets map[string]interface{} + Parallelism int + ProcessingGuarantees ProcessingGuarantees + RetainOrdering bool + Resources Resources + AutoAck bool + TimeoutMs int64 + Archive string + + // Whether the subscriptions the functions created/used should be deleted when the functions is deleted + CleanupSubscription bool + RuntimeFlags string +} diff --git a/pulsaradmin/pkg/pulsar/sourceConfig.go b/pulsaradmin/pkg/pulsar/sourceConfig.go new file mode 100644 index 0000000000..b0e80dc1ec --- /dev/null +++ b/pulsaradmin/pkg/pulsar/sourceConfig.go @@ -0,0 +1,45 @@ +// 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 pulsar + +type SourceConfig struct { + Tenant string + Namespace string + Name string + ClassName string + + TopicName string + SerdeClassName string + SchemaType string + + Configs map[string]interface{} + + // This is a map of secretName(aka how the secret is going to be + // accessed in the function via context) to an object that + // encapsulates how the secret is fetched by the underlying + // secrets provider. The type of an value here can be found by the + // SecretProviderConfigurator.getSecretObjectType() method. + Secrets map[string]interface{} + + Parallelism int + ProcessingGuarantees ProcessingGuarantees + Resources Resources + Archive string + // Any flags that you want to pass to the runtime. + RuntimeFlags string +} From 413d39d6a097e32c1c6421919da67da5d169f62a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Tue, 3 Sep 2019 13:38:48 +0800 Subject: [PATCH 093/348] Add stop cmd for Pulsar Functions (streamnative/pulsarctl#25) Signed-off-by: xiaolong.ran --- pulsaradmin/pkg/pulsar/data.go | 1 + pulsaradmin/pkg/pulsar/functions.go | 27 +++++++++++++++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index dbad0508d3..029d5382af 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -16,6 +16,7 @@ type FunctionData struct { Tenant string `json:"tenant"` Namespace string `json:"namespace"` FuncName string `json:"functionName"` + InstanceID string `json:"instance_id"` ClassName string `json:"className"` Jar string `json:"jarFile"` Py string `json:"pyFile"` diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index db09a619b6..d535203913 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -43,6 +43,12 @@ type Functions interface { // @param pkgUrl // url from which pkg can be downloaded CreateFuncWithUrl(data *FunctionConfig, pkgUrl string) error + + // Stop all function instances + StopFunction(tenant, namespace, name string) error + + // Stop function instance + StopFunctionWithID(tenant, namespace, name string, instanceID int) error } type functions struct { @@ -174,3 +180,24 @@ func (f *functions) CreateFuncWithUrl(funcConf *FunctionConfig, pkgUrl string) e return nil } + +func (f *functions) StopFunction(tenant, namespace, name string) error { + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) + err := f.client.post(endpoint+"/stop", "", nil) + if err != nil { + return err + } + return nil +} + +func (f *functions) StopFunctionWithID(tenant, namespace, name string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) + + err := f.client.post(endpoint+"/stop", "", nil) + if err != nil { + return err + } + + return nil +} From 24d877685fae0c10bc58d1037312988250a39233 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 4 Sep 2019 03:05:25 +0800 Subject: [PATCH 094/348] Add command cluster `create-failure-domain` (streamnative/pulsarctl#21) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Master issue: streamnative/pulsarctl#2 ``` USED FOR: This command is used for creating a failure domain of the . REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #creating the failure domain pulsarctl clusters create-failure-domain #creating the failure domain with brokers pulsarctl clusters create-failure-domain --broker-list --broker-list OUTPUT: #normal output Create failure domain for cluster succeed #the args need to be specified as [✖] need specified two names for cluster and failure domain #the specified cluster does not exist in the broker [✖] code: 404 reason: Cluster does not exist Usage: pulsarctl clusters create-failure-domain [flags] Aliases: create-failure-domain, cfd FailureDomainData flags: -b, --broker-list strings Set the failure domain clusters Common flags: -s, --admin-service-url string The admin web service url that pulsarctl connects to. (default "http://localhost:8080") --auth-params string Authentication parameters are used to configure the public and private key files required by tls For example: "tlsCertFile:val1,tlsKeyFile:val2" -C, --color string toggle colorized logs (true,false,fabulous) (default "true") -h, --help help for this command --tls-allow-insecure Allow TLS insecure connection --tls-trust-cert-pat string Allow TLS trust cert file path -v, --verbose int set log level, use 0 to silence, 4 for debugging (default 3) Use 'pulsarctl clusters create-failure-domain [command] --help' for more information about a command. ``` --- pulsaradmin/go.mod | 2 +- pulsaradmin/pkg/pulsar/cluster.go | 16 +++++++++++----- pulsaradmin/pkg/pulsar/data.go | 7 +++++++ 3 files changed, 19 insertions(+), 6 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index d2b82cd1ed..a4a4bece07 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -11,7 +11,7 @@ require ( github.com/mattn/go-colorable v0.1.2 // indirect github.com/mattn/go-runewidth v0.0.4 // indirect github.com/olekukonko/tablewriter v0.0.1 - github.com/pkg/errors v0.8.1 // indirect + github.com/pkg/errors v0.8.1 github.com/spf13/cobra v0.0.5 github.com/spf13/pflag v1.0.3 github.com/stretchr/objx v0.2.0 // indirect diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index 9465a9e917..a98b8cf387 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -10,6 +10,7 @@ type Clusters interface { Update(ClusterData) error UpdatePeerClusters(string, []string) error GetPeerClusters(string) ([]string, error) + CreateFailureDomain(FailureDomainData) error } type clusters struct { @@ -51,14 +52,19 @@ func (c *clusters) Update(cdata ClusterData) error { endpoint := c.client.endpoint(c.basePath, cdata.Name) return c.client.post(endpoint, &cdata, nil) } - -func (c *clusters) UpdatePeerClusters(cluster string, peerClusters []string) error { - endpoint := c.client.endpoint(c.basePath, cluster, "peers") - return c.client.post(endpoint, peerClusters, nil) -} func (c *clusters) GetPeerClusters(name string) ([]string, error) { var peerClusters []string endpoint := c.client.endpoint(c.basePath, name, "peers") err := c.client.get(endpoint, &peerClusters) return peerClusters, err } + +func (c *clusters) UpdatePeerClusters(cluster string, peerClusters []string) error { + endpoint := c.client.endpoint(c.basePath, cluster, "peers") + return c.client.post(endpoint, peerClusters, nil) +} + +func (c *clusters) CreateFailureDomain(data FailureDomainData) error { + endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) + return c.client.post(endpoint, &data, nil) +} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index 029d5382af..760c391f8e 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -50,3 +50,10 @@ type FunctionData struct { FuncConf *FunctionConfig `json:"-"` UserCodeFile string `json:"-"` } + +// Failure Domain information +type FailureDomainData struct { + ClusterName string `json:"-"` + DomainName string `json:"-"` + BrokerList []string `json:"brokerList"` +} From 83ae0df86a7f8ee8b0765ec649078f9101c3cb2e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Wed, 4 Sep 2019 03:07:03 +0800 Subject: [PATCH 095/348] Add delete cmd for Pulsar Functions (streamnative/pulsarctl#26) Signed-off-by: xiaolong.ran Master Issue: streamnative/pulsarctl#2 Add delete cmd for Pulsar Functions, the output as follows: ``` USED FOR: This command is used for delete a Pulsar Function that is running on a Pulsar cluster. REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #Delete a Pulsar Function that is running on a Pulsar cluster pulsarctl functions create --tenant public --namespace default --name #Delete a Pulsar Function that is running on a Pulsar cluster with instance ID pulsarctl functions create --tenant public --namespace default --name --instance-id 1 #Delete a Pulsar Function that is running on a Pulsar cluster with FQFN pulsarctl functions delete --fqfn tenant/namespace/name [eg: public/default/ExampleFunctions] OUTPUT: #normal output Deleted successfully Usage: pulsarctl functions delete [flags] Aliases: delete, delete FunctionsConfig flags: --fqfn string The Fully Qualified Function Name (FQFN) for the function --tenant string The tenant of a Pulsar Function --namespace string The namespace of a Pulsar Function --name string The name of a Pulsar Function ``` --- pulsaradmin/pkg/pulsar/functions.go | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index d535203913..a2d5555aa5 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -49,6 +49,8 @@ type Functions interface { // Stop function instance StopFunctionWithID(tenant, namespace, name string, instanceID int) error + + DeleteFunction(tenant, namespace, name string) error } type functions struct { @@ -183,21 +185,17 @@ func (f *functions) CreateFuncWithUrl(funcConf *FunctionConfig, pkgUrl string) e func (f *functions) StopFunction(tenant, namespace, name string) error { endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - err := f.client.post(endpoint+"/stop", "", nil) - if err != nil { - return err - } - return nil + return f.client.post(endpoint+"/stop", "", nil) } func (f *functions) StopFunctionWithID(tenant, namespace, name string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) - err := f.client.post(endpoint+"/stop", "", nil) - if err != nil { - return err - } + return f.client.post(endpoint+"/stop", "", nil) +} - return nil +func (f *functions) DeleteFunction(tenant, namespace, name string) error { + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) + return f.client.delete(endpoint, nil) } From 0df6b4fbba1b4090bf3717d8397e8ec4bf742671 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 4 Sep 2019 12:45:31 +0800 Subject: [PATCH 096/348] Add command cluster `get-failure-domain` (streamnative/pulsarctl#22) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Master issue: streamnative/pulsarctl#2 output example: ``` ➜ pulsarctl git:(clusters_get_failure_domain) ./pulsarctl cluster get-failure-domain -h USED FOR: This command is used for getting the specified failure domain on the specified cluster. REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #getting the broker list in the cluster failure domain pulsarctl clusters get-failure-domain -n OUTPUT: #output example { "brokers" : [ "failure-broker-A", "failure-broker-B", ] } #the cluster name is not specified or the cluster name is specified more than one [✖] only one argument is allowed to be used as a name #the specified cluster does not exist in the broker [✖] code: 404 reason: Cluster does not exist Usage: pulsarctl clusters get-failure-domain [flags] Aliases: get-failure-domain, gfd FailureDomain flags: -n, --domain-name string The failure domain name Common flags: -s, --admin-service-url string The admin web service url that pulsarctl connects to. (default "http://localhost:8080") -C, --color string toggle colorized logs (true,false,fabulous) (default "true") -h, --help help for this command -v, --verbose int set log level, use 0 to silence, 4 for debugging (default 3) Use 'pulsarctl clusters get-failure-domain [command] --help' for more information about a command. ``` --- pulsaradmin/pkg/pulsar/cluster.go | 9 +++++++++ pulsaradmin/pkg/pulsar/data.go | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index a98b8cf387..acdae4eb73 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -11,6 +11,7 @@ type Clusters interface { UpdatePeerClusters(string, []string) error GetPeerClusters(string) ([]string, error) CreateFailureDomain(FailureDomainData) error + GetFailureDomain(clusterName, domainName string) (FailureDomainData, error) } type clusters struct { @@ -52,6 +53,7 @@ func (c *clusters) Update(cdata ClusterData) error { endpoint := c.client.endpoint(c.basePath, cdata.Name) return c.client.post(endpoint, &cdata, nil) } + func (c *clusters) GetPeerClusters(name string) ([]string, error) { var peerClusters []string endpoint := c.client.endpoint(c.basePath, name, "peers") @@ -68,3 +70,10 @@ func (c *clusters) CreateFailureDomain(data FailureDomainData) error { endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) return c.client.post(endpoint, &data, nil) } + +func (c *clusters) GetFailureDomain(clusterName string, domainName string) (FailureDomainData, error) { + var res FailureDomainData + endpoint := c.client.endpoint(c.basePath, clusterName, "failureDomains", domainName) + err := c.client.get(endpoint, &res) + return res, err +} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index 760c391f8e..ede0d3a65e 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -55,5 +55,5 @@ type FunctionData struct { type FailureDomainData struct { ClusterName string `json:"-"` DomainName string `json:"-"` - BrokerList []string `json:"brokerList"` + BrokerList []string `json:"brokers"` } From ef44052fe570259eea35d74c05e92aa97700ddab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Wed, 4 Sep 2019 14:13:07 +0800 Subject: [PATCH 097/348] Add start cmd for Pulsar Functions (streamnative/pulsarctl#29) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Master Issue: streamnative/pulsarctl#2 Add start cmd for Pulsar Functions ``` USED FOR: This command is used for starting a stopped function instance. REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #Starts a stopped function instance pulsarctl functions start --tenant public --namespace default --name #Starts a stopped function instance with instance ID pulsarctl functions start --tenant public --namespace default --name --instance-id 1 #Starts a stopped function instance with FQFN pulsarctl functions start --fqfn tenant/namespace/name [eg: public/default/ExampleFunctions] OUTPUT: #normal output Started successfully #You must specify a name for the Pulsar Functions or a FQFN, please check the --name args [✖] you must specify a name for the function or a Fully Qualified Function Name (FQFN) #The name of Pulsar Functions doesn't exist, please check the --name args [✖] code: 404 reason: Function doesn't exist #Used an instanceID that does not exist or other impermissible actions [✖] code: 400 reason: Operation not permitted Usage: pulsarctl functions start [flags] Aliases: start, start FunctionsConfig flags: --fqfn string The Fully Qualified Function Name (FQFN) for the function --tenant string The tenant of a Pulsar Function --namespace string The namespace of a Pulsar Function --name string The name of a Pulsar Function --instance-id string The function instanceId (start all instances if instance-id is not provided) ``` --- pulsaradmin/pkg/pulsar/functions.go | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index a2d5555aa5..71815ca0e1 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -50,7 +50,14 @@ type Functions interface { // Stop function instance StopFunctionWithID(tenant, namespace, name string, instanceID int) error + // Delete an existing function DeleteFunction(tenant, namespace, name string) error + + // Start all function instances + StartFunction(tenant, namespace, name string) error + + // Start function instance + StartFunctionWithID(tenant, namespace, name string, instanceID int) error } type functions struct { @@ -199,3 +206,16 @@ func (f *functions) DeleteFunction(tenant, namespace, name string) error { endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) return f.client.delete(endpoint, nil) } + +func (f *functions) StartFunction(tenant, namespace, name string) error { + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) + return f.client.post(endpoint+"/start", "", nil) +} + +func (f *functions) StartFunctionWithID(tenant, namespace, name string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) + + return f.client.post(endpoint+"/start", "", nil) +} + From f5ca38df29a94847e78c3ceb3f6ef594eaa31150 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 4 Sep 2019 14:14:00 +0800 Subject: [PATCH 098/348] Add command cluster `list-failure-domains` (streamnative/pulsarctl#23) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Master issue: streamnative/pulsarctl#2 ``` ➜ pulsarctl git:(clusters_list_failure_domain) ./pulsarctl cluster list-failure-domains -h USED FOR: This command is used for getting all failure domain under the cluster . REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #listing all the failure domains under the specified cluster pulsarctl clusters list-failure-domains OUTPUT: #output example { "failure-domain": { "brokers": [ "failure-broker-A", "failure-broker-B" ] } } #the cluster name is not specified or the cluster name is specified more than one [✖] only one argument is allowed to be used as a name #the specified cluster does not exist in the broker [✖] code: 404 reason: Cluster does not exist Usage: pulsarctl clusters list-failure-domains [flags] Aliases: list-failure-domains, lfd Common flags: -s, --admin-service-url string The admin web service url that pulsarctl connects to. (default "http://localhost:8080") --auth-params string Authentication parameters are used to configure the public and private key files required by tls For example: "tlsCertFile:val1,tlsKeyFile:val2" -C, --color string toggle colorized logs (true,false,fabulous) (default "true") -h, --help help for this command --tls-allow-insecure Allow TLS insecure connection --tls-trust-cert-pat string Allow TLS trust cert file path -v, --verbose int set log level, use 0 to silence, 4 for debugging (default 3) Use 'pulsarctl clusters list-failure-domains [command] --help' for more information about a command. ``` --- pulsaradmin/pkg/pulsar/cluster.go | 8 ++++++++ pulsaradmin/pkg/pulsar/data.go | 2 ++ 2 files changed, 10 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index acdae4eb73..59ef3a66c9 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -12,6 +12,7 @@ type Clusters interface { GetPeerClusters(string) ([]string, error) CreateFailureDomain(FailureDomainData) error GetFailureDomain(clusterName, domainName string) (FailureDomainData, error) + ListFailureDomains(string) (FailureDomainMap, error) } type clusters struct { @@ -77,3 +78,10 @@ func (c *clusters) GetFailureDomain(clusterName string, domainName string) (Fail err := c.client.get(endpoint, &res) return res, err } + +func (c *clusters) ListFailureDomains(clusterName string) (FailureDomainMap, error) { + var domainData FailureDomainMap + endpoint := c.client.endpoint(c.basePath, clusterName, "failureDomains") + err := c.client.get(endpoint, &domainData) + return domainData, err +} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index ede0d3a65e..04924f35e0 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -57,3 +57,5 @@ type FailureDomainData struct { DomainName string `json:"-"` BrokerList []string `json:"brokers"` } + +type FailureDomainMap map[string]FailureDomainData From 1d49befaf24fd8560ad9e97e9498b05a6a70d9f9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Wed, 4 Sep 2019 15:42:06 +0800 Subject: [PATCH 099/348] Add restart, list and get cmd for Pulsar Functions (streamnative/pulsarctl#30) Signed-off-by: xiaolong.ran * Add restart cmd for Pulsar Functions Signed-off-by: xiaolong.ran * Add list cmd for Pulsar Functions Signed-off-by: xiaolong.ran * Add get cmd for Pulsar functions Signed-off-by: xiaolong.ran --- pulsaradmin/pkg/pulsar/functionConfg.go | 22 ++++----------- pulsaradmin/pkg/pulsar/functions.go | 37 +++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/sinkConfig.go | 2 +- pulsaradmin/pkg/pulsar/sourceConfig.go | 2 +- 4 files changed, 45 insertions(+), 18 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/functionConfg.go b/pulsaradmin/pkg/pulsar/functionConfg.go index 01c0ffdfbf..13eb356e36 100644 --- a/pulsaradmin/pkg/pulsar/functionConfg.go +++ b/pulsaradmin/pkg/pulsar/functionConfg.go @@ -17,20 +17,10 @@ package pulsar -type ProcessingGuarantees int - -type Runtime int - -const ( - AtLeasetOnce ProcessingGuarantees = iota - AtMostOnce - EffectivelyOnce -) - const ( - Java Runtime = iota - Python - Go + JavaRuntime = "JAVA" + PythonRuntime = "PYTHON" + GoRuntime = "GO" ) type FunctionConfig struct { @@ -58,7 +48,7 @@ type FunctionConfig struct { OutputSerdeClassName string `json:"outputSerdeClassName" yaml:"outputSerdeClassName"` LogTopic string `json:"logTopic" yaml:"logTopic"` - ProcessingGuarantees ProcessingGuarantees `json:"processingGuarantees" yaml:"processingGuarantees"` + ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` UserConfig map[string]interface{} `json:"userConfig" yaml:"userConfig"` @@ -69,7 +59,7 @@ type FunctionConfig struct { // SecretProviderConfigurator.getSecretObjectType() method. Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` - Runtime Runtime `json:"runtime" yaml:"runtime"` + Runtime string `json:"runtime" yaml:"runtime"` AutoAck bool `json:"autoAck" yaml:"autoAck"` MaxMessageRetries int `json:"maxMessageRetries" yaml:"maxMessageRetries"` DeadLetterTopic string `json:"deadLetterTopic" yaml:"deadLetterTopic"` @@ -78,7 +68,7 @@ type FunctionConfig struct { Resources *Resources `json:"resources" yaml:"resources"` FQFN string `json:"fqfn" yaml:"fqfn"` WindowConfig *WindowConfig `json:"windowConfig" yaml:"windowConfig"` - TimeoutMs *int64 `json:"timeoutMs" yaml:"timeoutMs"` + TimeoutMs *int64 `json:"timeoutMs" yaml:"timeoutMs"` Jar string `json:"jar" yaml:"jar"` Py string `json:"py" yaml:"py"` Go string `json:"go" yaml:"go"` diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index 71815ca0e1..089c8c5074 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -58,6 +58,18 @@ type Functions interface { // Start function instance StartFunctionWithID(tenant, namespace, name string, instanceID int) error + + // Restart all function instances + RestartFunction(tenant, namespace, name string) error + + // Restart function instance + RestartFunctionWithID(tenant, namespace, name string, instanceID int) error + + // Get the list of functions + GetFunctions(tenant, namespace string) ([]string, error) + + // Get the configuration for the specified function + GetFunction(tenant, namespace, name string) (FunctionConfig, error) } type functions struct { @@ -219,3 +231,28 @@ func (f *functions) StartFunctionWithID(tenant, namespace, name string, instance return f.client.post(endpoint+"/start", "", nil) } +func (f *functions) RestartFunction(tenant, namespace, name string) error { + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) + return f.client.post(endpoint+"/restart", "", nil) +} + +func (f *functions) RestartFunctionWithID(tenant, namespace, name string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) + + return f.client.post(endpoint+"/restart", "", nil) +} + +func (f *functions) GetFunctions(tenant, namespace string) ([]string, error) { + var functions []string + endpoint := f.client.endpoint(f.basePath, tenant, namespace) + err := f.client.get(endpoint, &functions) + return functions, err +} + +func (f *functions) GetFunction(tenant, namespace, name string) (FunctionConfig, error) { + var functionConfig FunctionConfig + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) + err := f.client.get(endpoint, &functionConfig) + return functionConfig, err +} diff --git a/pulsaradmin/pkg/pulsar/sinkConfig.go b/pulsaradmin/pkg/pulsar/sinkConfig.go index 75dc3cd082..db28bd568f 100644 --- a/pulsaradmin/pkg/pulsar/sinkConfig.go +++ b/pulsaradmin/pkg/pulsar/sinkConfig.go @@ -38,7 +38,7 @@ type SinkConfig struct { // SecretProviderConfigurator.getSecretObjectType() method. Secrets map[string]interface{} Parallelism int - ProcessingGuarantees ProcessingGuarantees + ProcessingGuarantees string RetainOrdering bool Resources Resources AutoAck bool diff --git a/pulsaradmin/pkg/pulsar/sourceConfig.go b/pulsaradmin/pkg/pulsar/sourceConfig.go index b0e80dc1ec..1a1b263a5c 100644 --- a/pulsaradmin/pkg/pulsar/sourceConfig.go +++ b/pulsaradmin/pkg/pulsar/sourceConfig.go @@ -37,7 +37,7 @@ type SourceConfig struct { Secrets map[string]interface{} Parallelism int - ProcessingGuarantees ProcessingGuarantees + ProcessingGuarantees string Resources Resources Archive string // Any flags that you want to pass to the runtime. From b480926377093ef3109fb7d2083b448e72ea5781 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Thu, 5 Sep 2019 14:11:19 +0800 Subject: [PATCH 100/348] Add update cmd for Pulsar Functions (streamnative/pulsarctl#33) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add update cmd for Pulsar Functions, output as follows: ``` USED FOR: Update a Pulsar Function that has been deployed to a Pulsar cluster. REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #Update output topic of Pulsar Function pulsarctl functions update --tenant public --namespace default --name update-function --output test-output-topic #Update function config yaml file of Pulsar Function pulsarctl functions update --function-config-file --jar #Update log topic of Pulsar Function pulsarctl functions update --log-topic persistent://public/default/test-log-topic # Other function parameters #Update dead letter topic of Pulsar Function pulsarctl functions update --dead-letter-topic persistent://public/default/test-dead-letter-topic --max-message-retries 10 # Other function parameters #Update user config of Pulsar Function pulsarctl functions update --user-config "{"publishTopic":"publishTopic", "key":"pulsar"}" # Other function parameters #Update custom schema of inputs topic for Pulsar Function pulsarctl functions update --custom-schema-inputs "{"topic-1":"schema.STRING", "topic-2":"schema.JSON"}" # Other function parameters #Update schema type of output topic for Pulsar Function pulsarctl functions update --schema-type schema.STRING # Other function parameters #Update parallelism of Pulsar Function pulsarctl functions update --parallelism 1 # Other function parameters #Update resource of Pulsar Function pulsarctl functions update --ram 5656565656 --disk 8080808080808080 --cpu 5.0 # Other function parameters #Update window functions config of Pulsar Function pulsarctl functions update --window-length-count 10 --window-length-duration-ms 1000 --sliding-interval-count 3 --sliding-interval-duration-ms 1000 # Other function parameters OUTPUT: #normal output Updated successfully #Update contains no change [✖] code: 400 reason: Update contains no change #The name of Pulsar Functions doesn't exist, please check the --name args [✖] code: 404 reason: Function doesn't exist Usage: pulsarctl functions update [flags] Aliases: update, update ``` --- pulsaradmin/pkg/pulsar/admin.go | 24 ++++ pulsaradmin/pkg/pulsar/data.go | 2 + pulsaradmin/pkg/pulsar/functions.go | 154 +++++++++++++++++++++++- pulsaradmin/pkg/pulsar/resources.go | 6 +- pulsaradmin/pkg/pulsar/updateOptions.go | 29 +++++ 5 files changed, 210 insertions(+), 5 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/updateOptions.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 53ca6bbd49..afa7ab5da5 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -223,6 +223,30 @@ func (c *client) post(endpoint string, in, obj interface{}) error { return nil } +func (c *client) putWithMultiPart(endpoint string, in, obj interface{}, body io.Reader, contentType string) error { + req, err := c.newRequest(http.MethodPut, endpoint) + if err != nil { + return err + } + req.obj = in + req.body = body + req.contentType = contentType + + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + + if obj != nil { + if err := decodeJsonBody(resp, &obj); err != nil { + return err + } + } + + return nil +} + func (c *client) postWithMultiPart(endpoint string, in, obj interface{}, body io.Reader, contentType string) error { req, err := c.newRequest(http.MethodPost, endpoint) if err != nil { diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index 04924f35e0..d73dc589e5 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -47,6 +47,8 @@ type FunctionData struct { MaxMessageRetries int `json:"maxMessageRetries"` DeadLetterTopic string `json:"deadLetterTopic"` + UpdateAuthData bool `json:"updateAuthData"` + FuncConf *FunctionConfig `json:"-"` UserCodeFile string `json:"-"` } diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index 089c8c5074..d72d3af2b7 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -70,6 +70,17 @@ type Functions interface { // Get the configuration for the specified function GetFunction(tenant, namespace, name string) (FunctionConfig, error) + + // Update the configuration for a function. + UpdateFunction(functionConfig *FunctionConfig, fileName string, updateOptions *UpdateOptions) error + + // Update the configuration for a function. + // + // Update a function by providing url from which fun-pkg can be downloaded. supported url: http/file + // eg: + // File: file:/dir/fileName.jar + // Http: http://www.repo.com/fileName.jar + UpdateFunctionWithUrl(functionConfig *FunctionConfig, pkgUrl string, updateOptions *UpdateOptions) error } type functions struct { @@ -86,14 +97,14 @@ func (c *client) Functions() Functions { func (f *functions) createStringFromField(w *multipart.Writer, value string) (io.Writer, error) { h := make(textproto.MIMEHeader) - h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, "functionConfig")) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) h.Set("Content-Type", "application/json") return w.CreatePart(h) } func (f *functions) createTextFromFiled(w *multipart.Writer, value string) (io.Writer, error) { h := make(textproto.MIMEHeader) - h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, "url")) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) h.Set("Content-Type", "text/plain") return w.CreatePart(h) } @@ -256,3 +267,142 @@ func (f *functions) GetFunction(tenant, namespace, name string) (FunctionConfig, err := f.client.get(endpoint, &functionConfig) return functionConfig, err } + +func (f *functions) UpdateFunction(functionConfig *FunctionConfig, fileName string, updateOptions *UpdateOptions) error { + endpoint := f.client.endpoint(f.basePath, functionConfig.Tenant, functionConfig.Namespace, functionConfig.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + jsonData, err := json.Marshal(functionConfig) + if err != nil { + return err + } + + stringWriter, err := f.createStringFromField(multiPartWriter, "functionConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if updateOptions != nil { + updateData, err := json.Marshal(updateOptions) + if err != nil { + return err + } + + updateStrWriter, err := f.createStringFromField(multiPartWriter, "updateOptions") + if err != nil { + return err + } + + _, err = updateStrWriter.Write(updateData) + if err != nil { + return err + } + } + + if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { + // If the function code is built in, we don't need to submit here + file, err := os.Open(fileName) + if err != nil { + return err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) + + if err != nil { + return err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = f.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} + +func (f *functions) UpdateFunctionWithUrl(functionConfig *FunctionConfig, pkgUrl string, updateOptions *UpdateOptions) error { + endpoint := f.client.endpoint(f.basePath, functionConfig.Tenant, functionConfig.Namespace, functionConfig.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + textWriter, err := f.createTextFromFiled(multiPartWriter, "url") + if err != nil { + return err + } + + _, err = textWriter.Write([]byte(pkgUrl)) + if err != nil { + return err + } + + jsonData, err := json.Marshal(functionConfig) + if err != nil { + return err + } + + stringWriter, err := f.createStringFromField(multiPartWriter, "functionConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if updateOptions != nil { + updateData, err := json.Marshal(updateOptions) + if err != nil { + return err + } + + updateStrWriter, err := f.createStringFromField(multiPartWriter, "updateOptions") + if err != nil { + return err + } + + _, err = updateStrWriter.Write(updateData) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = f.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} diff --git a/pulsaradmin/pkg/pulsar/resources.go b/pulsaradmin/pkg/pulsar/resources.go index 58730aad99..f1fd1431b7 100644 --- a/pulsaradmin/pkg/pulsar/resources.go +++ b/pulsaradmin/pkg/pulsar/resources.go @@ -18,9 +18,9 @@ package pulsar type Resources struct { - CPU float64 - Disk int64 - Ram int64 + CPU float64 `json:"cpu"` + Disk int64 `json:"disk"` + Ram int64 `json:"ram"` } func NewDefaultResources() *Resources { diff --git a/pulsaradmin/pkg/pulsar/updateOptions.go b/pulsaradmin/pkg/pulsar/updateOptions.go new file mode 100644 index 0000000000..dc93279fee --- /dev/null +++ b/pulsaradmin/pkg/pulsar/updateOptions.go @@ -0,0 +1,29 @@ +// 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 pulsar + +// Options while updating the sink +type UpdateOptions struct { + UpdateAuthData bool +} + +func NewUpdateOptions() *UpdateOptions { + return &UpdateOptions{ + UpdateAuthData: false, + } +} From d519e6b77dc9f2c6ef388bc69078cae89deb9e51 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 6 Sep 2019 00:10:54 +0800 Subject: [PATCH 101/348] Add command cluster `delete-failure-domain` (streamnative/pulsarctl#24) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Master issue: streamnative/pulsarctl#2 ``` ➜ pulsarctl git:(clusters_delete_failure_domain) ./pulsarctl cluster delete-failure-domain -h USED FOR: This command is used for deleting the failure domain of the cluster REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #deleting the failure domain pulsarctl clusters delete-failure-domain --domain-name OUTPUT: #output example Delete failure domain [] for cluster [] succeed #the cluster name is not specified or the cluster name is specified more than one [✖] only one argument is allowed to be used as a name #the specified cluster does not exist in the broker [✖] code: 404 reason: Cluster does not exist Usage: pulsarctl clusters delete-failure-domain [flags] Aliases: delete-failure-domain, dfd FailureDomainData flags: -n, --domain-name string The failure domain name Common flags: -s, --admin-service-url string The admin web service url that pulsarctl connects to. (default "http://localhost:8080") -C, --color string toggle colorized logs (true,false,fabulous) (default "true") -h, --help help for this command -v, --verbose int set log level, use 0 to silence, 4 for debugging (default 3) Use 'pulsarctl clusters delete-failure-domain [command] --help' for more information about a command. ``` --- pulsaradmin/pkg/pulsar/cluster.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index 59ef3a66c9..c99f54e809 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -13,6 +13,7 @@ type Clusters interface { CreateFailureDomain(FailureDomainData) error GetFailureDomain(clusterName, domainName string) (FailureDomainData, error) ListFailureDomains(string) (FailureDomainMap, error) + DeleteFailureDomain(FailureDomainData) error } type clusters struct { @@ -85,3 +86,8 @@ func (c *clusters) ListFailureDomains(clusterName string) (FailureDomainMap, err err := c.client.get(endpoint, &domainData) return domainData, err } + +func (c *clusters) DeleteFailureDomain(data FailureDomainData) error { + endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) + return c.client.delete(endpoint, nil) +} From 353031990031d56b51d68367476dbe82d62847d0 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 6 Sep 2019 10:40:56 +0800 Subject: [PATCH 102/348] Add namespace and topic name check (streamnative/pulsarctl#37) Add namespace and topic name check --- pulsaradmin/pkg/pulsar/namespace_name.go | 75 +++++++++++++ pulsaradmin/pkg/pulsar/namespace_name_test.go | 46 ++++++++ pulsaradmin/pkg/pulsar/topic_domain.go | 26 +++++ pulsaradmin/pkg/pulsar/topic_name.go | 102 ++++++++++++++++++ pulsaradmin/pkg/pulsar/topic_name_test.go | 48 +++++++++ 5 files changed, 297 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/namespace_name.go create mode 100644 pulsaradmin/pkg/pulsar/namespace_name_test.go create mode 100644 pulsaradmin/pkg/pulsar/topic_domain.go create mode 100644 pulsaradmin/pkg/pulsar/topic_name.go create mode 100644 pulsaradmin/pkg/pulsar/topic_name_test.go diff --git a/pulsaradmin/pkg/pulsar/namespace_name.go b/pulsaradmin/pkg/pulsar/namespace_name.go new file mode 100644 index 0000000000..b55a34d6d2 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/namespace_name.go @@ -0,0 +1,75 @@ +package pulsar + +import ( + "fmt" + "github.com/pkg/errors" + "regexp" + "strings" +) + +type NameSpaceName struct { + tenant string + nameSpace string +} + +func GetNameSpaceName(tenant, namespace string) (*NameSpaceName, error) { + return GetNamespaceName(fmt.Sprintf("%s/%s", tenant,namespace)) +} + +func GetNamespaceName(completeName string) (*NameSpaceName, error) { + var n NameSpaceName + + if completeName == "" { + return nil, errors.New("The namespace complete name is empty.") + } + + parts := strings.Split(completeName, "/") + if len(parts) == 2 { + n.tenant = parts[0] + n.nameSpace = parts[1] + err := validateNamespaceName(n.tenant, n.nameSpace) + if err != nil { + return nil, err + } + } else { + return nil, errors.Errorf("The complete name of namespace is invalid. complete name : [%s]", completeName) + } + + return &n, nil +} + +func (n *NameSpaceName) String() string { + return fmt.Sprintf("%s/%s", n.tenant, n.nameSpace) +} + +func validateNamespaceName(tenant, namespace string) error { + if tenant == "" || namespace == "" { + return errors.Errorf("Invalid tenant or namespace. [%s/%s]", tenant, namespace) + } + + ok := checkName(tenant) + if !ok { + return errors.Errorf("Tenant name include unsupported special chars. tenant : [%s]", tenant) + } + + ok = checkName(namespace) + if !ok { + return errors.Errorf("Namespace name include unsupported special chars. namespace : [%s]", namespace) + } + + return nil +} + +// allowed characters for property, namespace, cluster and topic +// names are alphanumeric (a-zA-Z0-9) and these special chars -=:. +// and % is allowed as part of valid URL encoding +const PATTEN = "^[-=:.\\w]*$" + +func checkName(name string) bool { + patten, err := regexp.Compile(PATTEN) + if err != nil { + return false + } + + return patten.MatchString(name) +} diff --git a/pulsaradmin/pkg/pulsar/namespace_name_test.go b/pulsaradmin/pkg/pulsar/namespace_name_test.go new file mode 100644 index 0000000000..8d1210cf62 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/namespace_name_test.go @@ -0,0 +1,46 @@ +package pulsar + +import ( + "github.com/stretchr/testify/assert" + "testing" +) + +func TestGetNamespaceName(t *testing.T) { + success, err := GetNamespaceName("public/default") + assert.Nil(t, err) + assert.Equal(t, "public/default", success.String()) + + empty, err := GetNamespaceName("") + assert.NotNil(t, err) + assert.Equal(t, "The namespace complete name is empty.", err.Error()) + assert.Nil(t, empty) + + empty, err = GetNamespaceName("/") + assert.NotNil(t, err) + assert.Equal(t, "Invalid tenant or namespace. [/]", err.Error()) + assert.Nil(t, empty) + + invalid, err := GetNamespaceName("public/default/fail") + assert.NotNil(t, err) + assert.Equal(t, "The complete name of namespace is invalid. complete name : [public/default/fail]", err.Error()) + assert.Nil(t, invalid) + + invalid, err = GetNamespaceName("public") + assert.NotNil(t, err) + assert.Equal(t, "The complete name of namespace is invalid. complete name : [public]", err.Error()) + assert.Nil(t, invalid) + + special, err := GetNamespaceName("-=.:/-=.:") + assert.Nil(t, err) + assert.Equal(t, "-=.:/-=.:", special.String()) + + tenantInvalid, err := GetNamespaceName("\"/namespace") + assert.NotNil(t, err) + assert.Equal(t, "Tenant name include unsupported special chars. tenant : [\"]", err.Error()) + assert.Nil(t, tenantInvalid) + + namespaceInvalid, err := GetNamespaceName("tenant/}") + assert.NotNil(t, err) + assert.Equal(t, "Namespace name include unsupported special chars. namespace : [}]", err.Error()) + assert.Nil(t, namespaceInvalid) +} diff --git a/pulsaradmin/pkg/pulsar/topic_domain.go b/pulsaradmin/pkg/pulsar/topic_domain.go new file mode 100644 index 0000000000..501ebcbe65 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/topic_domain.go @@ -0,0 +1,26 @@ +package pulsar + +import "github.com/pkg/errors" + +type TopicDomain string + +const ( + persistent TopicDomain = "persistent" + non_persistent TopicDomain = "non-persistent" +) + +func ParseTopicDomain(domain string) (TopicDomain, error) { + switch domain { + case "persistent": + return persistent, nil + case "non-persistent": + return non_persistent, nil + default: + return "", errors.Errorf("The domain only can be specified as 'persistent' or " + + "'non-persistent'. Input domain is '%s'.", domain) + } +} + +func (t TopicDomain) String() string { + return string(t) +} diff --git a/pulsaradmin/pkg/pulsar/topic_name.go b/pulsaradmin/pkg/pulsar/topic_name.go new file mode 100644 index 0000000000..78ee71f1af --- /dev/null +++ b/pulsaradmin/pkg/pulsar/topic_name.go @@ -0,0 +1,102 @@ +package pulsar + +import ( + "fmt" + "github.com/pkg/errors" + "strconv" + "strings" +) + +const ( + PUBLIC_TENANT = "public" + DEFAULT_NAMESPACE = "default" + PARTITIONED_TOPIC_SUFFIX = "-partition-" +) + +type TopicName struct { + domain TopicDomain + tenant string + namespace string + topic string + partitionIndex int + + namespaceName *NameSpaceName +} + +// The topic name can be in two different forms, one is fully qualified topic name, +// the other one is short topic name +func GetTopicName(completeName string) (*TopicName, error) { + var topicname TopicName + // The short topic name can be: + // - + // - // + if !strings.Contains(completeName, "://") { + parts := strings.Split(completeName, "/") + if len(parts) == 3 { + completeName = persistent.String() + "://" + completeName + } else if len(parts) == 1 { + completeName = persistent.String() + "://" + PUBLIC_TENANT + "/" + DEFAULT_NAMESPACE + "/" + parts[0] + } else { + return nil, errors.Errorf("Invalid short topic name '%s', it should be "+ + "in the format of // or ", completeName) + } + } + + // The fully qualified topic name can be: + // ://// + + parts := strings.Split(completeName, "://") + if len(parts) != 2 { + return nil, errors.Errorf("Invalid complete topic name '%s', it should be in "+ + "the format of :////", completeName) + } + + domain, err := ParseTopicDomain(parts[0]) + if err != nil { + return nil, err + } + topicname.domain = domain + + rest := parts[1] + parts = strings.Split(rest, "/") + if len(parts) == 3 { + topicname.tenant = parts[0] + topicname.namespace = parts[1] + topicname.topic = parts[2] + topicname.partitionIndex = getPartitionIndex(completeName) + } else { + return nil, errors.Errorf("Invalid topic name '%s', it should be in the format of "+ + "//", rest) + } + + n, err := GetNameSpaceName(topicname.tenant, topicname.namespace) + if err != nil { + return nil, err + } + topicname.namespaceName = n + + return &topicname, nil +} + +func (t *TopicName) String() string { + return fmt.Sprintf("%s://%s/%s/%s", t.domain, t.tenant, t.namespace, t.topic) +} + +func (t *TopicName) GetDomain() TopicDomain { + return t.domain +} + +func (t *TopicName) GetRestPath() string { + return fmt.Sprintf("%s/%s/%s/%s", t.domain, t.tenant, t.namespace, t.topic) +} + +func getPartitionIndex(topic string) int { + if strings.Contains(topic, PARTITIONED_TOPIC_SUFFIX) { + parts := strings.Split(topic, "-") + index, err := strconv.Atoi(parts[len(parts)-1]) + if err == nil { + return index + } + } + return -1 +} diff --git a/pulsaradmin/pkg/pulsar/topic_name_test.go b/pulsaradmin/pkg/pulsar/topic_name_test.go new file mode 100644 index 0000000000..46987efd60 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/topic_name_test.go @@ -0,0 +1,48 @@ +package pulsar + +import ( + "github.com/stretchr/testify/assert" + "testing" +) + +func TestGetTopicName(t *testing.T) { + success, err := GetTopicName("success") + assert.Nil(t, err) + assert.Equal(t, "persistent://public/default/success", success.String()) + + success, err = GetTopicName("tenant/namespace/success") + assert.Nil(t, err) + assert.Equal(t, "persistent://tenant/namespace/success", success.String()) + + success, err = GetTopicName("persistent://tenant/namespace/success") + assert.Nil(t, err) + assert.Equal(t, "persistent://tenant/namespace/success", success.String()) + + success, err = GetTopicName("non-persistent://tenant/namespace/success") + assert.Nil(t, err) + assert.Equal(t, "non-persistent://tenant/namespace/success", success.String()) + + fail, err := GetTopicName("default/fail") + assert.NotNil(t, err) + assert.Equal(t, "Invalid short topic name 'default/fail', it should be in the "+ + "format of // or ", err.Error()) + assert.Nil(t, fail) + + fail, err = GetTopicName("domain://tenant/namespace/fail") + assert.NotNil(t, err) + assert.Equal(t, "The domain only can be specified as 'persistent' or 'non-persistent'. "+ + "Input domain is 'domain'.", err.Error()) + assert.Nil(t, fail) + + fail, err = GetTopicName("persistent:///tenant/namespace/fail") + assert.NotNil(t, err) + assert.Equal(t, "Invalid topic name '/tenant/namespace/fail', it should be in the format "+ + "of //", err.Error()) + assert.Nil(t, fail) + + fail, err = GetTopicName("persistent://tenant/namespace") + assert.NotNil(t, err) + assert.Equal(t, "Invalid topic name 'tenant/namespace', it should be in the format "+ + "of //", err.Error()) + assert.Nil(t, fail) +} From a763f370a2ec636379638e02ecf0bafd56558bef Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 6 Sep 2019 16:15:46 +0800 Subject: [PATCH 103/348] Add more tests and add encode topic method (streamnative/pulsarctl#38) --- pulsaradmin/pkg/pulsar/topic_name.go | 19 +++++++++----- pulsaradmin/pkg/pulsar/topic_name_test.go | 32 ++++++++++++++++++++--- 2 files changed, 41 insertions(+), 10 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/topic_name.go b/pulsaradmin/pkg/pulsar/topic_name.go index 78ee71f1af..030fd28704 100644 --- a/pulsaradmin/pkg/pulsar/topic_name.go +++ b/pulsaradmin/pkg/pulsar/topic_name.go @@ -3,6 +3,7 @@ package pulsar import ( "fmt" "github.com/pkg/errors" + "net/url" "strconv" "strings" ) @@ -45,11 +46,7 @@ func GetTopicName(completeName string) (*TopicName, error) { // The fully qualified topic name can be: // ://// - parts := strings.Split(completeName, "://") - if len(parts) != 2 { - return nil, errors.Errorf("Invalid complete topic name '%s', it should be in "+ - "the format of :////", completeName) - } + parts := strings.SplitN(completeName, "://", 2) domain, err := ParseTopicDomain(parts[0]) if err != nil { @@ -58,7 +55,7 @@ func GetTopicName(completeName string) (*TopicName, error) { topicname.domain = domain rest := parts[1] - parts = strings.Split(rest, "/") + parts = strings.SplitN(rest, "/", 3) if len(parts) == 3 { topicname.tenant = parts[0] topicname.namespace = parts[1] @@ -69,6 +66,10 @@ func GetTopicName(completeName string) (*TopicName, error) { "//", rest) } + if topicname.topic == "" { + return nil, errors.New("Topic name can not be empty.") + } + n, err := GetNameSpaceName(topicname.tenant, topicname.namespace) if err != nil { return nil, err @@ -87,7 +88,11 @@ func (t *TopicName) GetDomain() TopicDomain { } func (t *TopicName) GetRestPath() string { - return fmt.Sprintf("%s/%s/%s/%s", t.domain, t.tenant, t.namespace, t.topic) + return fmt.Sprintf("%s/%s/%s/%s", t.domain, t.tenant, t.namespace, t.GetEncodedTopic()) +} + +func (t *TopicName) GetEncodedTopic() string { + return url.QueryEscape(t.topic) } func getPartitionIndex(topic string) int { diff --git a/pulsaradmin/pkg/pulsar/topic_name_test.go b/pulsaradmin/pkg/pulsar/topic_name_test.go index 46987efd60..7cb45a49cb 100644 --- a/pulsaradmin/pkg/pulsar/topic_name_test.go +++ b/pulsaradmin/pkg/pulsar/topic_name_test.go @@ -2,6 +2,7 @@ package pulsar import ( "github.com/stretchr/testify/assert" + "net/url" "testing" ) @@ -22,7 +23,11 @@ func TestGetTopicName(t *testing.T) { assert.Nil(t, err) assert.Equal(t, "non-persistent://tenant/namespace/success", success.String()) - fail, err := GetTopicName("default/fail") + fail, err := GetTopicName("://tenant.namespace.topic") + assert.NotNil(t, err) + assert.Equal(t, "The domain only can be specified as 'persistent' or 'non-persistent'. Input domain is ''.", err.Error()) + + fail, err = GetTopicName("default/fail") assert.NotNil(t, err) assert.Equal(t, "Invalid short topic name 'default/fail', it should be in the "+ "format of // or ", err.Error()) @@ -36,8 +41,7 @@ func TestGetTopicName(t *testing.T) { fail, err = GetTopicName("persistent:///tenant/namespace/fail") assert.NotNil(t, err) - assert.Equal(t, "Invalid topic name '/tenant/namespace/fail', it should be in the format "+ - "of //", err.Error()) + assert.Equal(t, "Invalid tenant or namespace. [/tenant]", err.Error()) assert.Nil(t, fail) fail, err = GetTopicName("persistent://tenant/namespace") @@ -45,4 +49,26 @@ func TestGetTopicName(t *testing.T) { assert.Equal(t, "Invalid topic name 'tenant/namespace', it should be in the format "+ "of //", err.Error()) assert.Nil(t, fail) + + fail, err = GetTopicName("persistent://tenant/namespace/") + assert.NotNil(t, err) + assert.Equal(t, "Topic name can not be empty.", err.Error()) + assert.Nil(t, fail) +} + +func TestTopicNameEncodeTest(t *testing.T) { + encodedName := "a%3Aen-in_in_business_content_item_20150312173022_https%5C%3A%2F%2Fin.news.example.com%2Fr" + rawName := "a:en-in_in_business_content_item_20150312173022_https\\://in.news.example.com/r" + + assert.Equal(t, encodedName, url.QueryEscape(rawName)) + o, err := url.QueryUnescape(encodedName) + assert.Nil(t, err) + assert.Equal(t, rawName, o) + + topicName, err := GetTopicName("persistent://prop/ns/" + rawName) + assert.Nil(t, err) + + assert.Equal(t, rawName, topicName.topic) + assert.Equal(t, encodedName, topicName.GetEncodedTopic()) + assert.Equal(t, "persistent/prop/ns/"+encodedName, topicName.GetRestPath()) } From 265c02677b8a6d71aef663884ce24b3274976932 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Sat, 7 Sep 2019 13:25:29 +0800 Subject: [PATCH 104/348] Add command cluster `update_failure_domain` (streamnative/pulsarctl#36) master issue: streamnative/pulsarctl#2 --- pulsaradmin/pkg/pulsar/cluster.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index c99f54e809..55661abd21 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -14,6 +14,7 @@ type Clusters interface { GetFailureDomain(clusterName, domainName string) (FailureDomainData, error) ListFailureDomains(string) (FailureDomainMap, error) DeleteFailureDomain(FailureDomainData) error + UpdateFailureDomain(FailureDomainData) error } type clusters struct { @@ -91,3 +92,7 @@ func (c *clusters) DeleteFailureDomain(data FailureDomainData) error { endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) return c.client.delete(endpoint, nil) } +func (c *clusters) UpdateFailureDomain(data FailureDomainData) error { + endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) + return c.client.post(endpoint, &data, nil) +} From 428fb7165da989f9e14c0108e0a5ffffbd61eb51 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 9 Sep 2019 14:44:22 +0800 Subject: [PATCH 105/348] Add tenants related commands (streamnative/pulsarctl#28) Master issue: streamnative/pulsarctl#2 The pull requests add all tenant commands. --- pulsaradmin/pkg/pulsar/Tenant.go | 50 ++++++++++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/admin.go | 1 + pulsaradmin/pkg/pulsar/data.go | 9 +++++- 3 files changed, 59 insertions(+), 1 deletion(-) create mode 100644 pulsaradmin/pkg/pulsar/Tenant.go diff --git a/pulsaradmin/pkg/pulsar/Tenant.go b/pulsaradmin/pkg/pulsar/Tenant.go new file mode 100644 index 0000000000..b41627b649 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/Tenant.go @@ -0,0 +1,50 @@ +package pulsar + +type Tenants interface { + Create(TenantData) error + Delete(string) error + Update(TenantData) error + List() ([]string, error) + Get(string) (TenantData, error) +} + +type tenants struct { + client *client + basePath string +} + +func (c *client) Tenants() Tenants { + return &tenants{ + client: c, + basePath: "/tenants", + } +} + +func (c *tenants) Create(data TenantData) error { + endpoint := c.client.endpoint(c.basePath, data.Name) + return c.client.put(endpoint, &data, nil) +} + +func (c *tenants) Delete(name string) error { + endpoint := c.client.endpoint(c.basePath, name) + return c.client.delete(endpoint, nil) +} + +func (c *tenants) Update(data TenantData) error { + endpoint := c.client.endpoint(c.basePath, data.Name) + return c.client.post(endpoint, &data, nil) +} + +func (c *tenants) List() ([]string, error) { + var tenantList []string + endpoint := c.client.endpoint(c.basePath, "") + err := c.client.get(endpoint, &tenantList) + return tenantList, err +} + +func (c *tenants) Get(name string) (TenantData, error) { + var data TenantData + endpoint := c.client.endpoint(c.basePath, name) + err := c.client.get(endpoint, &data) + return data, err +} diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index afa7ab5da5..81805008a9 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -54,6 +54,7 @@ func DefaultConfig() *Config { type Client interface { Clusters() Clusters Functions() Functions + Tenants() Tenants } type client struct { diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index d73dc589e5..a68c1f13a6 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -16,7 +16,7 @@ type FunctionData struct { Tenant string `json:"tenant"` Namespace string `json:"namespace"` FuncName string `json:"functionName"` - InstanceID string `json:"instance_id"` + InstanceID string `json:"instance_id"` ClassName string `json:"className"` Jar string `json:"jarFile"` Py string `json:"pyFile"` @@ -61,3 +61,10 @@ type FailureDomainData struct { } type FailureDomainMap map[string]FailureDomainData + +// Tenant args +type TenantData struct { + Name string `json:"-"` + AdminRoles []string `json:"adminRoles"` + AllowedClusters []string `json:"allowedClusters"` +} From 8cce397b76a0c95dd7066dc5dbb73519d85a758d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Mon, 9 Sep 2019 14:47:02 +0800 Subject: [PATCH 106/348] Add stats, status, querystate, putstate, trigger cmds for Pulsar Functions (streamnative/pulsarctl#34) Master Issue: streamnative/pulsarctl#2 Add `stats`, `status`, `querystate`, `putstate`, `trigger` cmds for Pulsar Functions --- pulsaradmin/pkg/pulsar/data.go | 24 +++ pulsaradmin/pkg/pulsar/function_state.go | 26 ++++ pulsaradmin/pkg/pulsar/function_status.go | 53 +++++++ pulsaradmin/pkg/pulsar/functions.go | 171 ++++++++++++++++++++++ pulsaradmin/pkg/pulsar/functions_stats.go | 145 ++++++++++++++++++ 5 files changed, 419 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/function_state.go create mode 100644 pulsaradmin/pkg/pulsar/function_status.go create mode 100644 pulsaradmin/pkg/pulsar/functions_stats.go diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index a68c1f13a6..16607d3970 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -1,3 +1,20 @@ +// 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 pulsar // ClusterData information on a cluster @@ -47,6 +64,13 @@ type FunctionData struct { MaxMessageRetries int `json:"maxMessageRetries"` DeadLetterTopic string `json:"deadLetterTopic"` + Key string `json:"key"` + Watch bool `json:"watch"` + State string `json:"state"` + TriggerValue string `json:"triggerValue"` + TriggerFile string `json:"triggerFile"` + Topic string `json:"topic"` + UpdateAuthData bool `json:"updateAuthData"` FuncConf *FunctionConfig `json:"-"` diff --git a/pulsaradmin/pkg/pulsar/function_state.go b/pulsaradmin/pkg/pulsar/function_state.go new file mode 100644 index 0000000000..a740e20e2e --- /dev/null +++ b/pulsaradmin/pkg/pulsar/function_state.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 pulsar + +type FunctionState struct { + Key string `json:"key"` + StringValue string `json:"stringValue"` + ByteValue []byte `json:"byteValue"` + NumValue int64 `json:"numberValue"` + Version int64 `json:"version"` +} diff --git a/pulsaradmin/pkg/pulsar/function_status.go b/pulsaradmin/pkg/pulsar/function_status.go new file mode 100644 index 0000000000..1c699beb20 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/function_status.go @@ -0,0 +1,53 @@ +// 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 pulsar + +type FunctionStatus struct { + NumInstances int `json:"numInstances"` + NumRunning int `json:"numRunning"` + Instances []FunctionInstanceStatus `json:"instances"` +} + +type FunctionInstanceStatus struct { + InstanceId int `json:"instanceId"` + Status FunctionInstanceStatusData `json:"status"` +} + +type FunctionInstanceStatusData struct { + Running bool `json:"running"` + Err string `json:"error"` + NumRestarts int64 `json:"numRestarts"` + NumReceived int64 `json:"numReceived"` + NumSuccessfullyProcessed int64 `json:"numSuccessfullyProcessed"` + NumUserExceptions int64 `json:"numUserExceptions"` + LatestUserExceptions []ExceptionInformation `json:"latestUserExceptions"` + NumSystemExceptions int64 `json:"numSystemExceptions"` + LatestSystemExceptions []ExceptionInformation `json:"latestSystemExceptions"` + AverageLatency float64 `json:"averageLatency"` + LastInvocationTime int64 `json:"lastInvocationTime"` + WorkerId string `json:"workerId"` +} + +type ExceptionInformation struct { + ExceptionString string `json:"exceptionString"` + TimestampMs int64 `json:"timestampMs"` +} + +func (fs *FunctionStatus) AddInstance(functionInstanceStatus FunctionInstanceStatus) { + fs.Instances = append(fs.Instances, functionInstanceStatus) +} diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index d72d3af2b7..b02e29d10f 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -71,6 +71,30 @@ type Functions interface { // Get the configuration for the specified function GetFunction(tenant, namespace, name string) (FunctionConfig, error) + // Gets the current status of a function + GetFunctionStatus(tenant, namespace, name string) (FunctionStatus, error) + + // Gets the current status of a function instance + GetFunctionStatusWithInstanceID(tenant, namespace, name string, instanceID int) (FunctionInstanceStatusData, error) + + // Gets the current stats of a function + GetFunctionStats(tenant, namespace, name string) (FunctionStats, error) + + // Gets the current stats of a function instance + GetFunctionStatsWithInstanceID(tenant, namespace, name string, instanceID int) (FunctionInstanceStatsData, error) + + // Fetch the current state associated with a Pulsar Function + // + // Response Example: + // { "value : 12, version : 2"} + GetFunctionState(tenant, namespace, name, key string) (FunctionState, error) + + // Puts the given state associated with a Pulsar Function + PutFunctionState(tenant, namespace, name string, state FunctionState) error + + // Triggers the function by writing to the input topic + TriggerFunction(tenant, namespace, name, topic, triggerValue, triggerFile string) (string, error) + // Update the configuration for a function. UpdateFunction(functionConfig *FunctionConfig, fileName string, updateOptions *UpdateOptions) error @@ -406,3 +430,150 @@ func (f *functions) UpdateFunctionWithUrl(functionConfig *FunctionConfig, pkgUrl return nil } + +func (f *functions) GetFunctionStatus(tenant, namespace, name string) (FunctionStatus, error) { + var functionStatus FunctionStatus + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) + err := f.client.get(endpoint+"/status", &functionStatus) + return functionStatus, err +} + +func (f *functions) GetFunctionStatusWithInstanceID(tenant, namespace, name string, instanceID int) (FunctionInstanceStatusData, error) { + var functionInstanceStatusData FunctionInstanceStatusData + id := fmt.Sprintf("%d", instanceID) + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) + err := f.client.get(endpoint+"/status", &functionInstanceStatusData) + return functionInstanceStatusData, err +} + +func (f *functions) GetFunctionStats(tenant, namespace, name string) (FunctionStats, error) { + var functionStats FunctionStats + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) + err := f.client.get(endpoint+"/stats", &functionStats) + return functionStats, err +} + +func (f *functions) GetFunctionStatsWithInstanceID(tenant, namespace, name string, instanceID int) (FunctionInstanceStatsData, error) { + var functionInstanceStatsData FunctionInstanceStatsData + id := fmt.Sprintf("%d", instanceID) + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) + err := f.client.get(endpoint+"/stats", &functionInstanceStatsData) + return functionInstanceStatsData, err +} + +func (f *functions)GetFunctionState(tenant, namespace, name, key string) (FunctionState, error) { + var functionState FunctionState + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, "state", key) + err := f.client.get(endpoint, &functionState) + return functionState, err +} + +func (f *functions) PutFunctionState(tenant, namespace, name string, state FunctionState) error { + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, "state", state.Key) + + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + stateData, err := json.Marshal(state) + + if err != nil { + return err + } + + stateWriter, err := f.createStringFromField(multiPartWriter, "state") + if err != nil { + return err + } + + _, err = stateWriter.Write(stateData) + + if err != nil { + return err + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + + err = f.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + + if err != nil { + return err + } + + return nil +} + +func (f *functions) TriggerFunction(tenant, namespace, name, topic, triggerValue, triggerFile string) (string, error) { + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, "trigger") + + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + if triggerFile != "" { + file, err := os.Open(triggerFile) + if err != nil { + return "", err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("dataStream", filepath.Base(file.Name())) + + if err != nil { + return "", err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return "", err + } + } + + if triggerValue != "" { + valueWriter, err := f.createTextFromFiled(multiPartWriter, "data") + if err != nil { + return "", err + } + + _, err = valueWriter.Write([]byte(triggerValue)) + if err != nil { + return "", err + } + } + + if topic != "" { + topicWriter, err := f.createTextFromFiled(multiPartWriter, "topic") + if err != nil { + return "", err + } + + _, err = topicWriter.Write([]byte(topic)) + if err != nil { + return "", err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err := multiPartWriter.Close(); err != nil { + return "", err + } + + contentType := multiPartWriter.FormDataContentType() + var str string + err := f.client.postWithMultiPart(endpoint, &str, nil, bodyBuf, contentType) + if err != nil { + return "", err + } + + return str, nil +} diff --git a/pulsaradmin/pkg/pulsar/functions_stats.go b/pulsaradmin/pkg/pulsar/functions_stats.go new file mode 100644 index 0000000000..3f644388b5 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/functions_stats.go @@ -0,0 +1,145 @@ +// 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 pulsar + +type FunctionStats struct { + // Overall total number of records function received from source + ReceivedTotal int64 `json:"receivedTotal"` + + // Overall total number of records successfully processed by user function + ProcessedSuccessfullyTotal int64 `json:"processedSuccessfullyTotal"` + + // Overall total number of system exceptions thrown + SystemExceptionsTotal int64 `json:"systemExceptionsTotal"` + + // Overall total number of user exceptions thrown + UserExceptionsTotal int64 `json:"userExceptionsTotal"` + + // Average process latency for function + AvgProcessLatency float64 `json:"avgProcessLatency"` + + // Timestamp of when the function was last invoked by any instance + LastInvocation int64 `json:"lastInvocation"` + + OneMin FunctionInstanceStatsDataBase `json:"oneMin"` + + Instances []FunctionInstanceStats `json:"instances"` + + FunctionInstanceStats +} + +type FunctionInstanceStats struct { + FunctionInstanceStatsDataBase + + InstanceId int64 `json:"instanceId"` + + Metrics FunctionInstanceStatsData `json:"metrics"` +} + +type FunctionInstanceStatsDataBase struct { + // Total number of records function received from source for instance + ReceivedTotal int64 `json:"receivedTotal"` + + // Total number of records successfully processed by user function for instance + ProcessedSuccessfullyTotal int64 `json:"processedSuccessfullyTotal"` + + // Total number of system exceptions thrown for instance + SystemExceptionsTotal int64 `json:"systemExceptionsTotal"` + + // Total number of user exceptions thrown for instance + UserExceptionsTotal int64 `json:"userExceptionsTotal"` + + // Average process latency for function for instance + AvgProcessLatency float64 `json:"avgProcessLatency"` +} + +type FunctionInstanceStatsData struct { + OneMin FunctionInstanceStatsDataBase `json:"oneMin"` + + // Timestamp of when the function was last invoked for instance + LastInvocation int64 `json:"lastInvocation"` + + // Map of user defined metrics + UserMetrics map[string]float64 `json:"userMetrics"` + + FunctionInstanceStatsDataBase +} + +func (fs *FunctionStats) AddInstance(functionInstanceStats FunctionInstanceStats) { + fs.Instances = append(fs.Instances, functionInstanceStats) +} + +func (fs *FunctionStats) CalculateOverall() *FunctionStats { + var ( + nonNullInstances int + nonNullInstancesOneMin int + ) + + for _, functionInstanceStats := range fs.Instances { + functionInstanceStatsData := functionInstanceStats.Metrics + fs.ReceivedTotal += functionInstanceStatsData.ReceivedTotal + fs.ProcessedSuccessfullyTotal += functionInstanceStatsData.ProcessedSuccessfullyTotal + fs.SystemExceptionsTotal += functionInstanceStatsData.SystemExceptionsTotal + fs.UserExceptionsTotal += functionInstanceStatsData.UserExceptionsTotal + + if functionInstanceStatsData.AvgProcessLatency != 0 { + if fs.AvgProcessLatency == 0 { + fs.AvgProcessLatency = 0.0 + } + + fs.AvgProcessLatency += functionInstanceStatsData.AvgProcessLatency + nonNullInstances++ + } + + fs.OneMin.ReceivedTotal += functionInstanceStatsData.OneMin.ReceivedTotal + fs.OneMin.ProcessedSuccessfullyTotal += functionInstanceStatsData.OneMin.ProcessedSuccessfullyTotal + fs.OneMin.SystemExceptionsTotal += functionInstanceStatsData.OneMin.SystemExceptionsTotal + fs.OneMin.UserExceptionsTotal += functionInstanceStatsData.OneMin.UserExceptionsTotal + + if functionInstanceStatsData.OneMin.AvgProcessLatency != 0 { + if fs.OneMin.AvgProcessLatency == 0 { + fs.OneMin.AvgProcessLatency = 0.0 + } + + fs.OneMin.AvgProcessLatency += functionInstanceStatsData.OneMin.AvgProcessLatency + nonNullInstancesOneMin++ + } + + if functionInstanceStatsData.LastInvocation != 0 { + if fs.LastInvocation == 0 || functionInstanceStatsData.LastInvocation > fs.LastInvocation { + fs.LastInvocation = functionInstanceStatsData.LastInvocation + } + } + } + + // calculate average from sum + if nonNullInstances > 0 { + fs.AvgProcessLatency = fs.AvgProcessLatency / float64(nonNullInstances) + } else { + fs.AvgProcessLatency = 0 + } + + // calculate 1min average from sum + if nonNullInstancesOneMin > 0 { + fs.OneMin.AvgProcessLatency = fs.OneMin.AvgProcessLatency / float64(nonNullInstancesOneMin) + } else { + fs.AvgProcessLatency = 0 + } + + return fs +} From 35f14cce94dd5978e26a7bc555068830ed54cc9e Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 11 Sep 2019 02:07:43 +0800 Subject: [PATCH 107/348] Add partitioned topic command CURD (streamnative/pulsarctl#35) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Master issue: streamnative/pulsarctl#2 *Modifications* Add commands topic `create`, `delete`, `update`, `get-metadata`, `list`. *TODO* need add test for these commands. ## OUTPUT * create * ``` ➜ pulsarctl-yong git:(topic-partitioned-commands) ./pulsarctl topic create -h USED FOR: This command is used for creating topic. REQUIRED PERMISSION: This command requires admin permissions. EXAMPLES: #Create non-partitioned topic pulsarctl topics create 0 #Create topic with partitions pulsarctl topics create OUTPUT: #normal output Create topic with partitions successfully #the topic name and(or) the partitions is not specified [✖] need to specified the topic name and the partitions #the topic has been created [✖] code: 409 reason: Partitioned topic already exists #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics create [flags] Aliases: create, c ``` * delete * ``` ➜ pulsarctl-yong git:(topic-partitioned-commands) ./pulsarctl topic delete -h USED FOR: This command is used for deleting an exist topic. REQUIRED PERMISSION: This command requires admin permissions. EXAMPLES: #Delete a partitioned topic pulsarctl topics delete #Delete a non-partitioned topic pulsarctl topics delete --non-partitioned OUTPUT: #normal output Delete topic successfully #the topic name is not specified [✖] only one argument is allowed to be used as a name #the partitioned topic does not exist [✖] code: 404 reason: Partitioned topic does not exist #the non-partitioned topic does not exist [✖] code: 404 reason: Topic not found #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics delete [flags] Aliases: delete, d ``` * get-metadata * ``` ➜ pulsarctl-yong git:(topic-partitioned-commands) ./pulsarctl topic get-metadata -h USED FOR: This command is used for getting an exist topic. REQUIRED PERMISSION: This command requires admin permissions. EXAMPLES: #Get an exist topic metadata pulsarctl topics get-metadata OUTPUT: #normal output { "partitions": "" } #the topic name is not specified [✖] only one argument is allowed to be used as a name #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics get-metadata [flags] Aliases: get-metadata, gm ``` * list * ``` ➜ pulsarctl-yong git:(topic-partitioned-commands) ./pulsarctl topic list -h USED FOR: This command is used for listing all exist topics under the specified namespace. REQUIRED PERMISSION: This command requires admin permissions. EXAMPLES: #List all exist topics under the namespace pulsarctl topics list OUTPUT: #normal output +--------------------------------+--------------------------------+ | PUBLIC/DEFAULT PARTITIONED | PUBLIC/DEFAULT NON-PARTITIONED | | TOPICS | TOPICS | +--------------------------------+--------------------------------+ | | | +--------------------------------+--------------------------------+ #the namespace is not specified [✖] only one argument is allowed to be used as a name #the tenant of the namespace is not exist [✖] code: 404 reason: Tenant does not exist #the namespace is not exist [✖] code: 404 reason: Namespace does not exist #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics list [flags] Aliases: list, lp ``` * update * ``` ➜ pulsarctl-yong git:(topic-partitioned-commands) ./pulsarctl topic update -h USED FOR: This command is used for updating an exist topic with new partition number. REQUIRED PERMISSION: This command requires admin permissions. EXAMPLES: # pulsarctl topics update OUTPUT: #normal output Update topic with partitions successfully #the topic name and(or) the partitions is not specified [✖] need to specified the topic name and the partitions #the partitions number is invalid [✖] invalid partition number '' #the topic is not exist [✖] code: 409 reason: Topic is not partitioned topic #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics update [flags] Aliases: update, up ``` --- pulsaradmin/pkg/pulsar/admin.go | 12 ++++ pulsaradmin/pkg/pulsar/data.go | 5 ++ pulsaradmin/pkg/pulsar/topic.go | 104 ++++++++++++++++++++++++++++++++ 3 files changed, 121 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/topic.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 81805008a9..6edcdd06c4 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -55,6 +55,7 @@ type Client interface { Clusters() Clusters Functions() Functions Tenants() Tenants + Topics() Topics } type client struct { @@ -185,11 +186,22 @@ func (c *client) put(endpoint string, in, obj interface{}) error { } func (c *client) delete(endpoint string, obj interface{}) error { + return c.deleteWithQueryParams(endpoint, obj, nil) +} + +func (c *client) deleteWithQueryParams(endpoint string, obj interface{}, params map[string]string) error { req, err := c.newRequest(http.MethodDelete, endpoint) if err != nil { return err } + if params != nil { + query := req.url.Query() + for k, v := range params { + query.Add(k, v) + } + } + resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return err diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index 16607d3970..2371b2df5a 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -92,3 +92,8 @@ type TenantData struct { AdminRoles []string `json:"adminRoles"` AllowedClusters []string `json:"allowedClusters"` } + +// Topic data +type PartitionedTopicMetadata struct { + Partitions int `json:"partitions"` +} diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go new file mode 100644 index 0000000000..4d5ed4e43c --- /dev/null +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -0,0 +1,104 @@ +package pulsar + +import ( + "strconv" +) + +type Topics interface { + Create(TopicName, int) error + Delete(TopicName, bool, bool) error + Update(TopicName, int) error + GetMetadata(TopicName) (PartitionedTopicMetadata, error) + List(NameSpaceName) ([]string, []string, error) +} + +type topics struct { + client *client + basePath string + persistentPath string + nonPersistentPath string +} + +func (c *client) Topics() Topics { + return &topics{ + client: c, + basePath: "", + persistentPath: "/persistent", + nonPersistentPath: "/non-persistent", + } +} + +func (t *topics) Create(topic TopicName, partitions int) error { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") + if partitions == 0 { + endpoint = t.client.endpoint(t.basePath, topic.GetRestPath()) + } + return t.client.put(endpoint, partitions, nil) +} + +func (t *topics) Delete(topic TopicName, force bool, nonPartitioned bool) error { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") + if nonPartitioned { + endpoint = t.client.endpoint(t.basePath, topic.GetRestPath()) + } + params := map[string]string{ + "force": strconv.FormatBool(force), + } + return t.client.deleteWithQueryParams(endpoint, nil, params) +} + +func (t *topics) Update(topic TopicName, partitions int) error { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") + return t.client.post(endpoint, partitions, nil) +} + +func (t *topics) GetMetadata(topic TopicName) (PartitionedTopicMetadata, error) { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") + var partitionedMeta PartitionedTopicMetadata + err := t.client.get(endpoint, &partitionedMeta) + return partitionedMeta, err +} + +func (t *topics) List(namespace NameSpaceName) ([]string, []string, error) { + var partitionedTopics, nonPartitionedTopics []string + partitionedTopicsChan := make(chan []string) + nonPartitionedTopicsChan := make(chan []string) + errChan := make(chan error) + + pp := t.client.endpoint(t.persistentPath, namespace.String(), "partitioned") + np := t.client.endpoint(t.nonPersistentPath, namespace.String(), "partitioned") + p := t.client.endpoint(t.persistentPath, namespace.String()) + n := t.client.endpoint(t.nonPersistentPath, namespace.String()) + + go t.getTopics(pp, partitionedTopicsChan, errChan) + go t.getTopics(np, partitionedTopicsChan, errChan) + go t.getTopics(p, nonPartitionedTopicsChan, errChan) + go t.getTopics(n, nonPartitionedTopicsChan, errChan) + + requestCount := 4 + for { + select { + case err :=<-errChan: + if err != nil { + return nil, nil, err + } + continue + case pTopic :=<- partitionedTopicsChan: + requestCount-- + partitionedTopics = append(partitionedTopics, pTopic...) + case npTopic :=<- nonPartitionedTopicsChan: + requestCount-- + nonPartitionedTopics = append(nonPartitionedTopics, npTopic...) + } + if requestCount == 0 { + break + } + } + return partitionedTopics, nonPartitionedTopics, nil +} + +func (t *topics) getTopics(endpoint string, out chan<- []string, err chan<- error) { + var topics []string + err <- t.client.get(endpoint, &topics) + out <- topics +} From 88b8ada325cd4e95e5fe8aa0ce61bac474913e47 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Wed, 11 Sep 2019 12:06:11 +0800 Subject: [PATCH 108/348] Add source command for pulsarctl (streamnative/pulsarctl#45) Signed-off-by: xiaolong.ran ranxiaolong716@gmail.com Add source command for pulsarctl - createSourcesCmd - updateSourcesCmd - deleteSourcesCmd - getSourcesCmd - listSourcesCmd - stopSourcesCmd - startSourcesCmd - restartSourcesCmd - statusSourcesCmd --- pulsaradmin/pkg/pulsar/admin.go | 1 + pulsaradmin/pkg/pulsar/connectorDefinition.go | 35 ++ pulsaradmin/pkg/pulsar/data.go | 24 + pulsaradmin/pkg/pulsar/sourceConfig.go | 28 +- pulsaradmin/pkg/pulsar/source_status.go | 59 +++ pulsaradmin/pkg/pulsar/sources.go | 433 ++++++++++++++++++ 6 files changed, 566 insertions(+), 14 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/connectorDefinition.go create mode 100644 pulsaradmin/pkg/pulsar/source_status.go create mode 100644 pulsaradmin/pkg/pulsar/sources.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 6edcdd06c4..2133c79144 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -55,6 +55,7 @@ type Client interface { Clusters() Clusters Functions() Functions Tenants() Tenants + Sources() Sources Topics() Topics } diff --git a/pulsaradmin/pkg/pulsar/connectorDefinition.go b/pulsaradmin/pkg/pulsar/connectorDefinition.go new file mode 100644 index 0000000000..4585e355da --- /dev/null +++ b/pulsaradmin/pkg/pulsar/connectorDefinition.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 pulsar + +// Basic information about a Pulsar connector +type ConnectorDefinition struct { + // The name of the connector type + Name string `json:"name"` + + // Description to be used for user help + Description string `json:"description"` + + // The class name for the connector source implementation + //

If not defined, it will be assumed this connector cannot act as a data source + SourceClass string `json:"sourceClass"` + + // The class name for the connector sink implementation + //

If not defined, it will be assumed this connector cannot act as a data sink + SinkClass string `json:"sinkClass"` +} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index 2371b2df5a..abd1249045 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -93,6 +93,30 @@ type TenantData struct { AllowedClusters []string `json:"allowedClusters"` } +type SourceData struct { + Tenant string `json:"tenant"` + Namespace string `json:"namespace"` + Name string `json:"name"` + SourceType string `json:"sourceType"` + ProcessingGuarantees string `json:"processingGuarantees"` + DestinationTopicName string `json:"destinationTopicName"` + DeserializationClassName string `json:"deserializationClassName"` + SchemaType string `json:"schemaType"` + Parallelism int `json:"parallelism"` + Archive string `json:"archive"` + ClassName string `json:"className"` + SourceConfigFile string `json:"sourceConfigFile"` + CPU float64 `json:"cpu"` + RAM int64 `json:"ram"` + Disk int64 `json:"disk"` + SourceConfigString string `json:"sourceConfigString"` + + SourceConf *SourceConfig `json:"-"` + InstanceID string `json:"instanceId"` + + UpdateAuthData bool `json:"updateAuthData"` +} + // Topic data type PartitionedTopicMetadata struct { Partitions int `json:"partitions"` diff --git a/pulsaradmin/pkg/pulsar/sourceConfig.go b/pulsaradmin/pkg/pulsar/sourceConfig.go index 1a1b263a5c..1405a5c587 100644 --- a/pulsaradmin/pkg/pulsar/sourceConfig.go +++ b/pulsaradmin/pkg/pulsar/sourceConfig.go @@ -18,28 +18,28 @@ package pulsar type SourceConfig struct { - Tenant string - Namespace string - Name string - ClassName string + Tenant string `json:"tenant" yaml:"tenant"` + Namespace string `json:"namespace" yaml:"namespace"` + Name string `json:"name" yaml:"name"` + ClassName string `json:"className" yaml:"className"` - TopicName string - SerdeClassName string - SchemaType string + TopicName string `json:"topicName" yaml:"topicName"` + SerdeClassName string `json:"serdeClassName" yaml:"serdeClassName"` + SchemaType string `json:"schemaType" yaml:"schemaType"` - Configs map[string]interface{} + Configs map[string]interface{} `json:"configs" yaml:"configs"` // This is a map of secretName(aka how the secret is going to be // accessed in the function via context) to an object that // encapsulates how the secret is fetched by the underlying // secrets provider. The type of an value here can be found by the // SecretProviderConfigurator.getSecretObjectType() method. - Secrets map[string]interface{} + Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` - Parallelism int - ProcessingGuarantees string - Resources Resources - Archive string + Parallelism int `json:"parallelism" yaml:"parallelism"` + ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` + Resources *Resources `json:"resources" yaml:"resources"` + Archive string `json:"archive" yaml:"archive"` // Any flags that you want to pass to the runtime. - RuntimeFlags string + RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` } diff --git a/pulsaradmin/pkg/pulsar/source_status.go b/pulsaradmin/pkg/pulsar/source_status.go new file mode 100644 index 0000000000..9c3833aee6 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/source_status.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 pulsar + +type SourceStatus struct { + NumInstances int `json:"numInstances"` + NumRunning int `json:"numRunning"` + + Instances []*SourceInstanceStatus `json:"instances"` +} + +type SourceInstanceStatus struct { + InstanceId int `json:"instanceId"` + + Status SourceInstanceStatusData `json:"status"` +} + +type SourceInstanceStatusData struct { + Running bool `json:"running"` + + Err string `json:"error"` + + NumRestarts int64 `json:"numRestarts"` + + NumReceivedFromSource int64 `json:"numReceivedFromSource"` + + NumSystemExceptions int64 `json:"numSystemExceptions"` + + LatestSystemExceptions []ExceptionInformation `json:"latestSystemExceptions"` + + NumSourceExceptions int64 `json:"numSourceExceptions"` + + LatestSourceExceptions []ExceptionInformation `json:"latestSourceExceptions"` + + NumWritten int64 `json:"numWritten"` + + LastReceivedTime int64 `json:"lastReceivedTime"` + + WorkerId string `json:"workerId"` +} + +func (ss *SourceStatus)AddInstance(sourceInstanceStatus *SourceInstanceStatus) { + ss.Instances = append(ss.Instances, sourceInstanceStatus) +} diff --git a/pulsaradmin/pkg/pulsar/sources.go b/pulsaradmin/pkg/pulsar/sources.go new file mode 100644 index 0000000000..5d3f9a1d40 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/sources.go @@ -0,0 +1,433 @@ +// 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 pulsar + +import ( + `bytes` + `encoding/json` + `fmt` + `io` + `mime/multipart` + `net/textproto` + `os` + `path/filepath` + `strings` +) + +type Sources interface { + // Get the list of all the Pulsar Sources. + ListSources(tenant, namespace string) ([]string, error) + + // Get the configuration for the specified source + GetSource(tenant, namespace, source string) (SourceConfig, error) + + // Create a new source + CreateSource(config *SourceConfig, fileName string) error + + // Create a new source by providing url from which fun-pkg can be downloaded. supported url: http/file + CreateSourceWithURL(config *SourceConfig, pkgUrl string) error + + // Update the configuration for a source. + UpdateSource(config *SourceConfig, fileName string, options *UpdateOptions) error + + // Update a source by providing url from which fun-pkg can be downloaded. supported url: http/file + UpdateSourceWithUrl(config *SourceConfig, pkgUrl string, options *UpdateOptions) error + + // Delete an existing source + DeleteSource(tenant, namespace, source string) error + + // Gets the current status of a source. + GetSourceStatus(tenant, namespace, source string) (SourceStatus, error) + + // Gets the current status of a source instance. + GetSourceStatusWithID(tenant, namespace, source string, id int) (SourceInstanceStatusData, error) + + // Restart all source instances + RestartSource(tenant, namespace, source string) error + + // Restart source instance + RestartSourceWithID(tenant, namespace, source string, id int) error + + // Stop all source instances + StopSource(tenant, namespace, source string) error + + // Stop source instance + StopSourceWithID(tenant, namespace, source string, id int) error + + // Start all source instances + StartSource(tenant, namespace, source string) error + + // Start source instance + StartSourceWithID(tenant, namespace, source string, id int) error + + // Fetches a list of supported Pulsar IO sources currently running in cluster mode + GetBuiltInSources() ([]*ConnectorDefinition, error) + + // Reload the available built-in connectors, include Source and Sink + ReloadBuiltInSources() error +} + +type sources struct { + client *client + basePath string +} + +func (c *client) Sources() Sources { + return &sources{ + client: c, + basePath: "/sources", + } +} + +func (s *sources) createStringFromField(w *multipart.Writer, value string) (io.Writer, error) { + h := make(textproto.MIMEHeader) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) + h.Set("Content-Type", "application/json") + return w.CreatePart(h) +} + +func (s *sources) createTextFromFiled(w *multipart.Writer, value string) (io.Writer, error) { + h := make(textproto.MIMEHeader) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) + h.Set("Content-Type", "text/plain") + return w.CreatePart(h) +} + +func (s *sources) ListSources(tenant, namespace string) ([]string, error) { + var sources []string + endpoint := s.client.endpoint(s.basePath, tenant, namespace) + err := s.client.get(endpoint, &sources) + return sources, err +} + +func (s *sources) GetSource(tenant, namespace, source string) (SourceConfig, error) { + var sourceConfig SourceConfig + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + err := s.client.get(endpoint, &sourceConfig) + return sourceConfig, err +} + +func (s *sources) CreateSource(config *SourceConfig, fileName string) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { + // If the function code is built in, we don't need to submit here + file, err := os.Open(fileName) + if err != nil { + return err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) + + if err != nil { + return err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} + +func (s *sources) CreateSourceWithURL(config *SourceConfig, pkgUrl string) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + textWriter, err := s.createTextFromFiled(multiPartWriter, "url") + if err != nil { + return err + } + + _, err = textWriter.Write([]byte(pkgUrl)) + if err != nil { + return err + } + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} + +func (s *sources) UpdateSource(config *SourceConfig, fileName string, updateOptions *UpdateOptions) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if updateOptions != nil { + updateData, err := json.Marshal(updateOptions) + if err != nil { + return err + } + + updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") + if err != nil { + return err + } + + _, err = updateStrWriter.Write(updateData) + if err != nil { + return err + } + } + + if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { + // If the function code is built in, we don't need to submit here + file, err := os.Open(fileName) + if err != nil { + return err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) + + if err != nil { + return err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} + +func (s *sources) UpdateSourceWithUrl(config *SourceConfig, pkgUrl string, updateOptions *UpdateOptions) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + textWriter, err := s.createTextFromFiled(multiPartWriter, "url") + if err != nil { + return err + } + + _, err = textWriter.Write([]byte(pkgUrl)) + if err != nil { + return err + } + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if updateOptions != nil { + updateData, err := json.Marshal(updateOptions) + if err != nil { + return err + } + + updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") + if err != nil { + return err + } + + _, err = updateStrWriter.Write(updateData) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} + +func (s *sources) DeleteSource(tenant, namespace, source string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + return s.client.delete(endpoint, nil) +} + +func (s *sources) GetSourceStatus(tenant, namespace, source string) (SourceStatus, error) { + var sourceStatus SourceStatus + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + err := s.client.get(endpoint+"/status", &sourceStatus) + return sourceStatus, err +} + +func (s *sources) GetSourceStatusWithID(tenant, namespace, source string, id int) (SourceInstanceStatusData, error) { + var sourceInstanceStatusData SourceInstanceStatusData + instanceID := fmt.Sprintf("%d", id) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, instanceID) + err := s.client.get(endpoint+"/status", &sourceInstanceStatusData) + return sourceInstanceStatusData, err +} + +func (s *sources) RestartSource(tenant, namespace, source string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + return s.client.post(endpoint+"/restart", "", nil) +} + +func (s *sources) RestartSourceWithID(tenant, namespace, source string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) + + return s.client.post(endpoint+"/restart", "", nil) +} + +func (s *sources) StopSource(tenant, namespace, source string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + return s.client.post(endpoint+"/stop", "", nil) +} + +func (s *sources) StopSourceWithID(tenant, namespace, source string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) + + return s.client.post(endpoint+"/stop", "", nil) +} + +func (s *sources) StartSource(tenant, namespace, source string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + return s.client.post(endpoint+"/start", "", nil) +} + +func (s *sources) StartSourceWithID(tenant, namespace, source string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) + + return s.client.post(endpoint+"/start", "", nil) +} + +func (s *sources) GetBuiltInSources() ([]*ConnectorDefinition, error) { + var connectorDefinition []*ConnectorDefinition + endpoint := s.client.endpoint(s.basePath, "builtinsources") + err := s.client.get(endpoint, &connectorDefinition) + return connectorDefinition, err +} + +func (s *sources) ReloadBuiltInSources() error { + endpoint := s.client.endpoint(s.basePath, "reloadBuiltInSources") + return s.client.post(endpoint, "", nil) +} From 1086ee3c0e8e3827a45391617b4e125ba8431c4d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Wed, 11 Sep 2019 20:37:36 +0800 Subject: [PATCH 109/348] [issue:50] fix response status code error (streamnative/pulsarctl#53) * [issue:50] fix response status code error Signed-off-by: xiaolong.ran --- pulsaradmin/pkg/pulsar/admin.go | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 2133c79144..f55d437df0 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -405,17 +405,12 @@ func responseError(resp *http.Response) error { return e } - jsonErr := json.Unmarshal(body, &e) + json.Unmarshal(body, &e) - if jsonErr != nil { - e.Code = http.StatusPartialContent - } else { - e.Code = resp.StatusCode - - if e.Reason == "" { - e.Reason = unknownErrorReason - } - } + e.Code = resp.StatusCode + if e.Reason == "" { + e.Reason = unknownErrorReason + } return e } From 5db1e694c7f4d95023bdeeb77daf7aa794df96bc Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Sat, 14 Sep 2019 04:19:35 +0800 Subject: [PATCH 110/348] Add command topic `lookup` and `bundle-range` (streamnative/pulsarctl#44) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## OUTPUT *lookup* ``` ➜ pulsarctl-yong git:(topic-lookup) ✗ ./pulsarctl topic lookup -h USED FOR: This command is used for looking up a topic. REQUIRED PERMISSION: This command does not require permissions. EXAMPLES: #Look up a topic pulsarctl topic lookup OUTPUT: # { "brokerUlr": "", "brokerUrlTls": "", "httpUrl": "", "httpUrlTls": "", } #the topic name is not specified [✖] only one argument is allowed to be used as a name #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics lookup [flags] ``` *bundle-range* ``` ➜ pulsarctl-yong git:(topic-lookup) ✗ ./pulsarctl topic bundle-range -h USED FOR: This command is used for getting namespace bundle range of a topic. REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #Get namespace bundle range of a topic pulsarctl topic bundle-range OUTPUT: #normal output The bundle range of the topic is: #the topic name is not specified [✖] only one argument is allowed to be used as a name #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics bundle-range [flags] Aliases: bundle-range, ``` --- pulsaradmin/pkg/pulsar/admin.go | 21 ++++++++++++++++----- pulsaradmin/pkg/pulsar/data.go | 7 +++++++ pulsaradmin/pkg/pulsar/topic.go | 26 ++++++++++++++++++++++---- 3 files changed, 45 insertions(+), 9 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index f55d437df0..bebd857270 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -143,25 +143,36 @@ func (c *client) endpoint(componentPath string, parts ...string) string { // get is used to do a GET request against an endpoint // and deserialize the response into an interface -func (c *client) get(endpoint string, obj interface{}) error { +func (c *client) getAndDecode(endpoint string, obj interface{}, decode bool) ([]byte, error) { req, err := c.newRequest(http.MethodGet, endpoint) if err != nil { - return err + return nil, err } resp, err := checkSuccessful(c.doRequest(req)) if err != nil { - return err + return nil, err } defer safeRespClose(resp) if obj != nil { if err := decodeJsonBody(resp, &obj); err != nil { - return err + return nil, err } + } else if !decode { + body, err := ioutil.ReadAll(resp.Body) + if err != nil { + return nil, err + } + return body, err } - return nil + return nil, err +} + +func (c *client) get(endpoint string, obj interface{}) error { + _, err := c.getAndDecode(endpoint, obj, true) + return err } func (c *client) put(endpoint string, in, obj interface{}) error { diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index abd1249045..e9d717de63 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -121,3 +121,10 @@ type SourceData struct { type PartitionedTopicMetadata struct { Partitions int `json:"partitions"` } + +type LookupData struct { + BrokerUrl string `json:"brokerUrl"` + BrokerUrlTls string `json:"brokerUrlTls"` + HttpUrl string `json:"httpUrl"` + HttpUrlTls string `json:"httpUrlTls"` +} \ No newline at end of file diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 4d5ed4e43c..365c4b617c 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -1,6 +1,7 @@ package pulsar import ( + "fmt" "strconv" ) @@ -10,6 +11,8 @@ type Topics interface { Update(TopicName, int) error GetMetadata(TopicName) (PartitionedTopicMetadata, error) List(NameSpaceName) ([]string, []string, error) + Lookup(TopicName) (LookupData, error) + GetBundleRange(TopicName) (string, error) } type topics struct { @@ -17,6 +20,7 @@ type topics struct { basePath string persistentPath string nonPersistentPath string + lookupPath string } func (c *client) Topics() Topics { @@ -25,6 +29,7 @@ func (c *client) Topics() Topics { basePath: "", persistentPath: "/persistent", nonPersistentPath: "/non-persistent", + lookupPath: "/lookup/v2/topic", } } @@ -61,7 +66,7 @@ func (t *topics) GetMetadata(topic TopicName) (PartitionedTopicMetadata, error) func (t *topics) List(namespace NameSpaceName) ([]string, []string, error) { var partitionedTopics, nonPartitionedTopics []string - partitionedTopicsChan := make(chan []string) + partitionedTopicsChan := make(chan []string) nonPartitionedTopicsChan := make(chan []string) errChan := make(chan error) @@ -78,15 +83,15 @@ func (t *topics) List(namespace NameSpaceName) ([]string, []string, error) { requestCount := 4 for { select { - case err :=<-errChan: + case err := <-errChan: if err != nil { return nil, nil, err } continue - case pTopic :=<- partitionedTopicsChan: + case pTopic := <-partitionedTopicsChan: requestCount-- partitionedTopics = append(partitionedTopics, pTopic...) - case npTopic :=<- nonPartitionedTopicsChan: + case npTopic := <-nonPartitionedTopicsChan: requestCount-- nonPartitionedTopics = append(nonPartitionedTopics, npTopic...) } @@ -102,3 +107,16 @@ func (t *topics) getTopics(endpoint string, out chan<- []string, err chan<- erro err <- t.client.get(endpoint, &topics) out <- topics } + +func (t *topics) Lookup(topic TopicName) (LookupData, error) { + var lookup LookupData + endpoint := fmt.Sprintf("%s/%s", t.lookupPath, topic.GetRestPath()) + err := t.client.get(endpoint, &lookup) + return lookup, err +} + +func (t *topics) GetBundleRange(topic TopicName) (string, error) { + endpoint := fmt.Sprintf("%s/%s/%s", t.lookupPath, topic.GetRestPath(), "bundle") + data, err := t.client.getAndDecode(endpoint, nil, false) + return string(data), err +} From 848ab635ddd7098297834cc18cbc66414019e34d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Sat, 14 Sep 2019 17:33:11 +0800 Subject: [PATCH 111/348] Add schemas commands for pulsarctl (streamnative/pulsarctl#52) * Add schemas commands foor pulsarctl Signed-off-by: xiaolong.ran --- pulsaradmin/pkg/pulsar/admin.go | 7 +- pulsaradmin/pkg/pulsar/data.go | 13 ++- pulsaradmin/pkg/pulsar/schema.go | 133 ++++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/schema_util.go | 69 +++++++++++++ pulsaradmin/pkg/pulsar/topic_name.go | 6 +- 5 files changed, 223 insertions(+), 5 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/schema.go create mode 100644 pulsaradmin/pkg/pulsar/schema_util.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index bebd857270..21b58e3dd8 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -27,9 +27,9 @@ type Config struct { HttpClient *http.Client ApiVersion ApiVersion - Auth *auth.TlsAuthProvider - AuthParams string - TlsOptions *TLSOptions + Auth *auth.TlsAuthProvider + AuthParams string + TlsOptions *TLSOptions } type TLSOptions struct { @@ -57,6 +57,7 @@ type Client interface { Tenants() Tenants Sources() Sources Topics() Topics + Schemas() Schema } type client struct { diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index e9d717de63..ddc9a83fb0 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -122,9 +122,20 @@ type PartitionedTopicMetadata struct { Partitions int `json:"partitions"` } +type SchemaData struct { + Version int64 `json:"version"` + Filename string `json:"filename"` + Jar string `json:"jar"` + Type string `json:"type"` + Classname string `json:"classname"` + AlwaysAllowNull bool `json:"alwaysAllowNull"` + DryRun bool `json:"dryRun"` +} + type LookupData struct { BrokerUrl string `json:"brokerUrl"` BrokerUrlTls string `json:"brokerUrlTls"` HttpUrl string `json:"httpUrl"` HttpUrlTls string `json:"httpUrlTls"` -} \ No newline at end of file +} + diff --git a/pulsaradmin/pkg/pulsar/schema.go b/pulsaradmin/pkg/pulsar/schema.go new file mode 100644 index 0000000000..494acea332 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/schema.go @@ -0,0 +1,133 @@ +// 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 pulsar + +import ( + "fmt" + "strconv" +) + +type Schema interface { + // Retrieve the latest schema of a topic + GetSchemaInfo(topic string) (*SchemaInfo, error) + + // Retrieve the latest schema with version of a topic + GetSchemaInfoWithVersion(topic string) (*SchemaInfoWithVersion, error) + + // Retrieve the schema of a topic at a given version + GetSchemaInfoByVersion(topic string, version int64) (*SchemaInfo, error) + + // Delete the schema associated with a given topic + DeleteSchema(topic string) error + + // Create a schema for a given topic + CreateSchemaByPayload(topic string, schemaPayload PostSchemaPayload) error +} + +type schemas struct { + client *client + basePath string +} + +func (c *client) Schemas() Schema { + return &schemas{ + client: c, + basePath: "/schemas", + } +} + +func (s *schemas) GetSchemaInfo(topic string) (*SchemaInfo, error) { + topicName, err := GetTopicName(topic) + if err != nil { + return nil, err + } + var response GetSchemaResponse + endpoint := s.client.endpoint(s.basePath, topicName.tenant, topicName.namespace, topicName.GetEncodedTopic(), "schema") + + err = s.client.get(endpoint, &response) + if err != nil { + return nil, err + } + + info := convertGetSchemaResponseToSchemaInfo(topicName, response) + return info, nil +} + +func (s *schemas) GetSchemaInfoWithVersion(topic string) (*SchemaInfoWithVersion, error) { + topicName, err := GetTopicName(topic) + if err != nil { + return nil, err + } + var response GetSchemaResponse + endpoint := s.client.endpoint(s.basePath, topicName.tenant, topicName.namespace, + topicName.GetEncodedTopic(), "schema") + + err = s.client.get(endpoint, &response) + if err != nil { + fmt.Println("err:", err.Error()) + return nil, err + } + + info := convertGetSchemaResponseToSchemaInfoWithVersion(topicName, response) + return info, nil +} + +func (s *schemas) GetSchemaInfoByVersion(topic string, version int64) (*SchemaInfo, error) { + topicName, err := GetTopicName(topic) + if err != nil { + return nil, err + } + + var response GetSchemaResponse + endpoint := s.client.endpoint(s.basePath, topicName.tenant, topicName.namespace, topicName.GetEncodedTopic(), + "schema", strconv.FormatInt(version, 10)) + + err = s.client.get(endpoint, &response) + if err != nil { + return nil, err + } + + info := convertGetSchemaResponseToSchemaInfo(topicName, response) + return info, nil +} + +func (s *schemas) DeleteSchema(topic string) error { + topicName, err := GetTopicName(topic) + if err != nil { + return err + } + + endpoint := s.client.endpoint(s.basePath, topicName.tenant, topicName.namespace, + topicName.GetEncodedTopic(), "schema") + + fmt.Println(endpoint) + + return s.client.delete(endpoint, nil) +} + +func (s *schemas) CreateSchemaByPayload(topic string, schemaPayload PostSchemaPayload) error { + topicName, err := GetTopicName(topic) + if err != nil { + return err + } + + endpoint := s.client.endpoint(s.basePath, topicName.tenant, topicName.namespace, + topicName.GetEncodedTopic(), "schema") + + return s.client.post(endpoint, &schemaPayload, nil) +} diff --git a/pulsaradmin/pkg/pulsar/schema_util.go b/pulsaradmin/pkg/pulsar/schema_util.go new file mode 100644 index 0000000000..08c3dab875 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/schema_util.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 pulsar + +type SchemaInfo struct { + Name string `json:"name"` + Schema []byte `json:"schema"` + Type string `json:"type"` + Properties map[string]string `json:"properties"` +} + +type SchemaInfoWithVersion struct { + Version int64 `json:"version"` + SchemaInfo *SchemaInfo `json:"schemaInfo"` +} + +// Payload with information about a schema +type PostSchemaPayload struct { + SchemaType string `json:"type"` + Schema string `json:"schema"` + Properties map[string]string `json:"properties"` +} + +type GetSchemaResponse struct { + Version int64 `json:"version"` + Type string `json:"type"` + Timestamp int64 `json:"timestamp"` + Data string `json:"data"` + Properties map[string]string `json:"properties"` +} + +func convertGetSchemaResponseToSchemaInfo(tn *TopicName, response GetSchemaResponse) *SchemaInfo { + info := new(SchemaInfo) + schema := make([]byte, 0, 10) + if response.Type == "KEY_VALUE" { + // TODO: impl logic + } else { + schema = []byte(response.Data) + } + + info.Schema = schema + info.Type = response.Type + info.Properties = response.Properties + info.Name = tn.GetLocalName() + + return info +} + +func convertGetSchemaResponseToSchemaInfoWithVersion(tn *TopicName, response GetSchemaResponse) *SchemaInfoWithVersion { + info := new(SchemaInfoWithVersion) + info.SchemaInfo = convertGetSchemaResponseToSchemaInfo(tn, response) + info.Version = response.Version + return info +} diff --git a/pulsaradmin/pkg/pulsar/topic_name.go b/pulsaradmin/pkg/pulsar/topic_name.go index 030fd28704..e955bc9f90 100644 --- a/pulsaradmin/pkg/pulsar/topic_name.go +++ b/pulsaradmin/pkg/pulsar/topic_name.go @@ -46,7 +46,7 @@ func GetTopicName(completeName string) (*TopicName, error) { // The fully qualified topic name can be: // ://// - parts := strings.SplitN(completeName, "://", 2) + parts := strings.SplitN(completeName, "://", 2) domain, err := ParseTopicDomain(parts[0]) if err != nil { @@ -95,6 +95,10 @@ func (t *TopicName) GetEncodedTopic() string { return url.QueryEscape(t.topic) } +func (t *TopicName) GetLocalName() string { + return t.topic +} + func getPartitionIndex(topic string) int { if strings.Contains(topic, PARTITIONED_TOPIC_SUFFIX) { parts := strings.Split(topic, "-") From 50fe6e2589c20b3a579d2a64366fee5eb9bedb2b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Mon, 16 Sep 2019 19:40:32 +0800 Subject: [PATCH 112/348] Add sink commands for pulsarctl (streamnative/pulsarctl#47) Add sink commands for pulsarctl - createSinksCmd - updateSinksCmd - deleteSinksCmd - getSinksCmd - listSinksCmd - stopSinksCmd - startSinksCmd - restartSinksCmd - statusSinksCmd --- pulsaradmin/pkg/pulsar/admin.go | 1 + pulsaradmin/pkg/pulsar/data.go | 29 ++ pulsaradmin/pkg/pulsar/sinkConfig.go | 42 +-- pulsaradmin/pkg/pulsar/sink_status.go | 72 +++++ pulsaradmin/pkg/pulsar/sinks.go | 434 ++++++++++++++++++++++++++ 5 files changed, 557 insertions(+), 21 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/sink_status.go create mode 100644 pulsaradmin/pkg/pulsar/sinks.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 21b58e3dd8..3b8348201b 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -56,6 +56,7 @@ type Client interface { Functions() Functions Tenants() Tenants Sources() Sources + Sinks() Sinks Topics() Topics Schemas() Schema } diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index ddc9a83fb0..ff0fe38f8d 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -117,6 +117,35 @@ type SourceData struct { UpdateAuthData bool `json:"updateAuthData"` } +type SinkData struct { + Tenant string `json:"tenant"` + Namespace string `json:"namespace"` + Name string `json:"name"` + SinkType string `json:"sinkType"` + Inputs string `json:"inputs"` + TopicsPattern string `json:"topicsPattern"` + SubsName string `json:"subsName"` + CustomSerdeInputString string `json:"customSerdeInputString"` + CustomSchemaInputString string `json:"customSchemaInputString"` + ProcessingGuarantees string `json:"processingGuarantees"` + RetainOrdering bool `json:"retainOrdering"` + Parallelism int `json:"parallelism"` + Archive string `json:"archive"` + ClassName string `json:"className"` + SinkConfigFile string `json:"sinkConfigFile"` + CPU float64 `json:"cpu"` + RAM int64 `json:"ram"` + Disk int64 `json:"disk"` + SinkConfigString string `json:"sinkConfigString"` + AutoAck bool `json:"autoAck"` + TimeoutMs int64 `json:"timeoutMs"` + + SinkConf *SinkConfig `json:"-"` + InstanceID string `json:"instanceId"` + + UpdateAuthData bool `json:"updateAuthData"` +} + // Topic data type PartitionedTopicMetadata struct { Partitions int `json:"partitions"` diff --git a/pulsaradmin/pkg/pulsar/sinkConfig.go b/pulsaradmin/pkg/pulsar/sinkConfig.go index db28bd568f..bf77168daf 100644 --- a/pulsaradmin/pkg/pulsar/sinkConfig.go +++ b/pulsaradmin/pkg/pulsar/sinkConfig.go @@ -18,34 +18,34 @@ package pulsar type SinkConfig struct { - Tenant string - Namespace string - Name string - ClassName string + Tenant string `json:"tenant" yaml:"tenant"` + Namespace string `json:"namespace" yaml:"namespace"` + Name string `json:"name" yaml:"name"` + ClassName string `json:"className" yaml:"className"` - SourceSubscriptionName string - Inputs []string - TopicToSerdeClassName map[string]string - TopicsPattern string - TopicToSchemaType map[string]string - InputSpecs map[string]ConsumerConfig - Configs map[string]interface{} + SourceSubscriptionName string `json:"sourceSubscriptionName" yaml:"sourceSubscriptionName"` + Inputs []string `json:"inputs" yaml:"inputs"` + TopicToSerdeClassName map[string]string `json:"topicToSerdeClassName" yaml:"topicToSerdeClassName"` + TopicsPattern *string `json:"topicsPattern" yaml:"topicsPattern"` + TopicToSchemaType map[string]string `json:"topicToSchemaType" yaml:"topicToSchemaType"` + InputSpecs map[string]ConsumerConfig `json:"inputSpecs" yaml:"inputSpecs"` + Configs map[string]interface{} `json:"configs" yaml:"configs"` // This is a map of secretName(aka how the secret is going to be // accessed in the function via context) to an object that // encapsulates how the secret is fetched by the underlying // secrets provider. The type of an value here can be found by the // SecretProviderConfigurator.getSecretObjectType() method. - Secrets map[string]interface{} - Parallelism int - ProcessingGuarantees string - RetainOrdering bool - Resources Resources - AutoAck bool - TimeoutMs int64 - Archive string + Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` + Parallelism int `json:"parallelism" yaml:"parallelism"` + ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` + RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` + Resources *Resources `json:"resources" yaml:"resources"` + AutoAck bool `json:"autoAck" yaml:"autoAck"` + TimeoutMs *int64 `json:"timeoutMs" yaml:"timeoutMs"` + Archive string `json:"archive" yaml:"archive"` // Whether the subscriptions the functions created/used should be deleted when the functions is deleted - CleanupSubscription bool - RuntimeFlags string + CleanupSubscription bool `json:"cleanupSubscription" yaml:"cleanupSubscription"` + RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` } diff --git a/pulsaradmin/pkg/pulsar/sink_status.go b/pulsaradmin/pkg/pulsar/sink_status.go new file mode 100644 index 0000000000..67c79be0f8 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/sink_status.go @@ -0,0 +1,72 @@ +// 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 pulsar + +type SinkStatus struct { + // The total number of sink instances that ought to be running + NumInstances int `json:"numInstances"` + // The number of source instances that are actually running + NumRunning int `json:"numRunning"` + + Instances []*SinkInstanceStatus `json:"instances"` +} + +type SinkInstanceStatus struct { + InstanceId int `json:"instanceId"` + + Status SourceInstanceStatusData `json:"status"` +} + +type SinkInstanceStatusData struct { + // Is this instance running? + Running bool `json:"running"` + + // Do we have any error while running this instance + Err string `json:"error"` + + // Number of times this instance has restarted + NumRestarts int64 `json:"numRestarts"` + + // Number of messages read from Pulsar + NumReadFromPulsar int64 `json:"numReadFromPulsar"` + + // Number of times there was a system exception handling messages + NumSystemExceptions int64 `json:"numSystemExceptions"` + + // A list of the most recent system exceptions + LatestSystemExceptions []ExceptionInformation `json:"latestSystemExceptions"` + + // Number of times there was a sink exception + NumSinkExceptions int64 `json:"numSinkExceptions"` + + // A list of the most recent sink exceptions + LatestSinkExceptions []ExceptionInformation `json:"latestSinkExceptions"` + + // Number of messages written to sink + NumWrittenToSink int64 `json:"numWrittenToSink"` + + // When was the last time we received a message from Pulsar + LastReceivedTime int64 `json:"lastReceivedTime"` + + WorkerId string `json:"workerId"` +} + +func (ss *SinkStatus)AddInstance(sinkInstanceStatus *SinkInstanceStatus) { + ss.Instances = append(ss.Instances, sinkInstanceStatus) +} + diff --git a/pulsaradmin/pkg/pulsar/sinks.go b/pulsaradmin/pkg/pulsar/sinks.go new file mode 100644 index 0000000000..6a31798674 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/sinks.go @@ -0,0 +1,434 @@ +// 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 pulsar + +import ( + `bytes` + `encoding/json` + `fmt` + `io` + `mime/multipart` + `net/textproto` + `os` + `path/filepath` + `strings` +) + +type Sinks interface { + // Get the list of all the Pulsar Sinks. + ListSinks(tenant, namespace string) ([]string, error) + + // Get the configuration for the specified sink + GetSink(tenant, namespace, Sink string) (SinkConfig, error) + + // Create a new sink + CreateSink(config *SinkConfig, fileName string) error + + // Create a new sink by providing url from which fun-pkg can be downloaded. supported url: http/file + CreateSinkWithURL(config *SinkConfig, pkgUrl string) error + + // Update the configuration for a sink. + UpdateSink(config *SinkConfig, fileName string, options *UpdateOptions) error + + // Update a sink by providing url from which fun-pkg can be downloaded. supported url: http/file + UpdateSinkWithUrl(config *SinkConfig, pkgUrl string, options *UpdateOptions) error + + // Delete an existing sink + DeleteSink(tenant, namespace, Sink string) error + + // Gets the current status of a sink. + GetSinkStatus(tenant, namespace, Sink string) (SinkStatus, error) + + // Gets the current status of a sink instance. + GetSinkStatusWithID(tenant, namespace, Sink string, id int) (SinkInstanceStatusData, error) + + // Restart all sink instances + RestartSink(tenant, namespace, Sink string) error + + // Restart sink instance + RestartSinkWithID(tenant, namespace, Sink string, id int) error + + // Stop all sink instances + StopSink(tenant, namespace, Sink string) error + + // Stop sink instance + StopSinkWithID(tenant, namespace, Sink string, id int) error + + // Start all sink instances + StartSink(tenant, namespace, Sink string) error + + // Start sink instance + StartSinkWithID(tenant, namespace, Sink string, id int) error + + // Fetches a list of supported Pulsar IO sinks currently running in cluster mode + GetBuiltInSinks() ([]*ConnectorDefinition, error) + + // Reload the available built-in connectors, include Source and Sink + ReloadBuiltInSinks() error +} + +type sinks struct { + client *client + basePath string +} + +func (c *client) Sinks() Sinks { + return &sinks{ + client: c, + basePath: "/sinks", + } +} + +func (s *sinks) createStringFromField(w *multipart.Writer, value string) (io.Writer, error) { + h := make(textproto.MIMEHeader) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) + h.Set("Content-Type", "application/json") + return w.CreatePart(h) +} + +func (s *sinks) createTextFromFiled(w *multipart.Writer, value string) (io.Writer, error) { + h := make(textproto.MIMEHeader) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) + h.Set("Content-Type", "text/plain") + return w.CreatePart(h) +} + +func (s *sinks) ListSinks(tenant, namespace string) ([]string, error) { + var sinks []string + endpoint := s.client.endpoint(s.basePath, tenant, namespace) + err := s.client.get(endpoint, &sinks) + return sinks, err +} + +func (s *sinks) GetSink(tenant, namespace, Sink string) (SinkConfig, error) { + var sinkConfig SinkConfig + endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) + err := s.client.get(endpoint, &sinkConfig) + return sinkConfig, err +} + +func (s *sinks) CreateSink(config *SinkConfig, fileName string) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { + // If the function code is built in, we don't need to submit here + file, err := os.Open(fileName) + if err != nil { + return err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) + + if err != nil { + return err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} + +func (s *sinks) CreateSinkWithURL(config *SinkConfig, pkgUrl string) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + textWriter, err := s.createTextFromFiled(multiPartWriter, "url") + if err != nil { + return err + } + + _, err = textWriter.Write([]byte(pkgUrl)) + if err != nil { + return err + } + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} + +func (s *sinks) UpdateSink(config *SinkConfig, fileName string, updateOptions *UpdateOptions) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if updateOptions != nil { + updateData, err := json.Marshal(updateOptions) + if err != nil { + return err + } + + updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") + if err != nil { + return err + } + + _, err = updateStrWriter.Write(updateData) + if err != nil { + return err + } + } + + if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { + // If the function code is built in, we don't need to submit here + file, err := os.Open(fileName) + if err != nil { + return err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) + + if err != nil { + return err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} + +func (s *sinks) UpdateSinkWithUrl(config *SinkConfig, pkgUrl string, updateOptions *UpdateOptions) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + textWriter, err := s.createTextFromFiled(multiPartWriter, "url") + if err != nil { + return err + } + + _, err = textWriter.Write([]byte(pkgUrl)) + if err != nil { + return err + } + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if updateOptions != nil { + updateData, err := json.Marshal(updateOptions) + if err != nil { + return err + } + + updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") + if err != nil { + return err + } + + _, err = updateStrWriter.Write(updateData) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} + +func (s *sinks) DeleteSink(tenant, namespace, Sink string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) + return s.client.delete(endpoint, nil) +} + +func (s *sinks) GetSinkStatus(tenant, namespace, Sink string) (SinkStatus, error) { + var sinkStatus SinkStatus + endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) + err := s.client.get(endpoint+"/status", &sinkStatus) + return sinkStatus, err +} + +func (s *sinks) GetSinkStatusWithID(tenant, namespace, Sink string, id int) (SinkInstanceStatusData, error) { + var sinkInstanceStatusData SinkInstanceStatusData + instanceID := fmt.Sprintf("%d", id) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink, instanceID) + err := s.client.get(endpoint+"/status", &sinkInstanceStatusData) + return sinkInstanceStatusData, err +} + +func (s *sinks) RestartSink(tenant, namespace, Sink string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) + return s.client.post(endpoint+"/restart", "", nil) +} + +func (s *sinks) RestartSinkWithID(tenant, namespace, Sink string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink, id) + + return s.client.post(endpoint+"/restart", "", nil) +} + +func (s *sinks) StopSink(tenant, namespace, Sink string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) + return s.client.post(endpoint+"/stop", "", nil) +} + +func (s *sinks) StopSinkWithID(tenant, namespace, Sink string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink, id) + + return s.client.post(endpoint+"/stop", "", nil) +} + +func (s *sinks) StartSink(tenant, namespace, Sink string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) + return s.client.post(endpoint+"/start", "", nil) +} + +func (s *sinks) StartSinkWithID(tenant, namespace, Sink string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink, id) + + return s.client.post(endpoint+"/start", "", nil) +} + +func (s *sinks) GetBuiltInSinks() ([]*ConnectorDefinition, error) { + var connectorDefinition []*ConnectorDefinition + endpoint := s.client.endpoint(s.basePath, "builtinSinks") + err := s.client.get(endpoint, &connectorDefinition) + return connectorDefinition, err +} + +func (s *sinks) ReloadBuiltInSinks() error { + endpoint := s.client.endpoint(s.basePath, "reloadBuiltInSinks") + return s.client.post(endpoint, "", nil) +} + From 66ed42075a48fdcf62c3e793e572538811f97069 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 17 Sep 2019 07:31:28 +0800 Subject: [PATCH 113/348] Add command topic `last-message-id` (streamnative/pulsarctl#42) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Master Issue: streamnative/pulsarctl#2 ##OUTPUT *last-message-id* ``` ➜ pulsarctl git:(topic-lastmessage) ./pulsarctl topic last-message-id -h USED FOR: This command is used for getting the last message id of a topic (partition). REQUIRED PERMISSION: This command requires tenant admin permissions. EXAMPLES: #Get the last message id of a topic pulsarctl topic last-message-id #Get the last message id of a partitioned topic pulsarctl topic last-message-id --partition-index OUTPUT: #normal output { "LedgerId": 0, "EntryId": 0, "PartitionedIndex": 0 } #the topic name is not specified [✖] only one argument is allowed to be used as a name #the topic does not exist in the cluster [✖] code: 404 reason: Topic not found #the topic does not a persistent topic [✖] code: 405 reason: GetLastMessageId on a non-persistent topic is not allowed #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics last-message-id [flags] Aliases: last-message-id, lmi ``` --- pulsaradmin/pkg/pulsar/message_id.go | 7 +++++++ pulsaradmin/pkg/pulsar/topic.go | 8 ++++++++ pulsaradmin/pkg/pulsar/topic_name.go | 13 +++++++++++++ 3 files changed, 28 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/message_id.go diff --git a/pulsaradmin/pkg/pulsar/message_id.go b/pulsaradmin/pkg/pulsar/message_id.go new file mode 100644 index 0000000000..c24aabd4c7 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/message_id.go @@ -0,0 +1,7 @@ +package pulsar + +type MessageId struct { + LedgerId int64 + EntryId int64 + PartitionedIndex int +} diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 365c4b617c..12377ebfc7 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -13,6 +13,7 @@ type Topics interface { List(NameSpaceName) ([]string, []string, error) Lookup(TopicName) (LookupData, error) GetBundleRange(TopicName) (string, error) + GetLastMessageId(TopicName) (MessageId, error) } type topics struct { @@ -120,3 +121,10 @@ func (t *topics) GetBundleRange(topic TopicName) (string, error) { data, err := t.client.getAndDecode(endpoint, nil, false) return string(data), err } + +func (t *topics) GetLastMessageId(topic TopicName) (MessageId, error) { + var messageId MessageId + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "lastMessageId") + err := t.client.get(endpoint, &messageId) + return messageId, err +} diff --git a/pulsaradmin/pkg/pulsar/topic_name.go b/pulsaradmin/pkg/pulsar/topic_name.go index e955bc9f90..5561c2b431 100644 --- a/pulsaradmin/pkg/pulsar/topic_name.go +++ b/pulsaradmin/pkg/pulsar/topic_name.go @@ -99,6 +99,19 @@ func (t *TopicName) GetLocalName() string { return t.topic } +func (t *TopicName) GetPartition(index int) (*TopicName, error) { + if index < 0 { + return nil, errors.New("Invalid partition index number.") + } + + if strings.Contains(t.String(), PARTITIONED_TOPIC_SUFFIX) { + return t, nil + } + + topicNameWithPartition := t.String() + PARTITIONED_TOPIC_SUFFIX + strconv.Itoa(index) + return GetTopicName(topicNameWithPartition) +} + func getPartitionIndex(topic string) int { if strings.Contains(topic, PARTITIONED_TOPIC_SUFFIX) { parts := strings.Split(topic, "-") From 24257e4bac97157fc4544476a6bb8187ce4fc5d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Tue, 17 Sep 2019 10:38:33 +0800 Subject: [PATCH 114/348] Add create, delete, list, policies, topic commands for namespace ctl (streamnative/pulsarctl#57) * Add create, delete, list, police, topic commands for namespace ctl Signed-off-by: xiaolong.ran --- pulsaradmin/pkg/pulsar/admin.go | 9 +- pulsaradmin/pkg/pulsar/auth_polices.go | 40 +++++ pulsaradmin/pkg/pulsar/backlog_quota.go | 50 +++++++ pulsaradmin/pkg/pulsar/bundles_data.go | 43 ++++++ pulsaradmin/pkg/pulsar/data.go | 12 +- pulsaradmin/pkg/pulsar/dispatch_rate.go | 44 ++++++ pulsaradmin/pkg/pulsar/namespace.go | 140 ++++++++++++++++++ .../pkg/pulsar/persistence_policies.go | 34 +++++ pulsaradmin/pkg/pulsar/policies.go | 82 ++++++++++ pulsaradmin/pkg/pulsar/retention_policies.go | 23 +++ pulsaradmin/pkg/pulsar/schema_strategy.go | 31 ++++ 11 files changed, 500 insertions(+), 8 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/auth_polices.go create mode 100644 pulsaradmin/pkg/pulsar/backlog_quota.go create mode 100644 pulsaradmin/pkg/pulsar/bundles_data.go create mode 100644 pulsaradmin/pkg/pulsar/dispatch_rate.go create mode 100644 pulsaradmin/pkg/pulsar/namespace.go create mode 100644 pulsaradmin/pkg/pulsar/persistence_policies.go create mode 100644 pulsaradmin/pkg/pulsar/policies.go create mode 100644 pulsaradmin/pkg/pulsar/retention_policies.go create mode 100644 pulsaradmin/pkg/pulsar/schema_strategy.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 3b8348201b..c4ea1c533b 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -58,6 +58,7 @@ type Client interface { Sources() Sources Sinks() Sinks Topics() Topics + Namespaces() Namespaces Schemas() Schema } @@ -420,10 +421,10 @@ func responseError(resp *http.Response) error { json.Unmarshal(body, &e) - e.Code = resp.StatusCode - if e.Reason == "" { - e.Reason = unknownErrorReason - } + e.Code = resp.StatusCode + if e.Reason == "" { + e.Reason = unknownErrorReason + } return e } diff --git a/pulsaradmin/pkg/pulsar/auth_polices.go b/pulsaradmin/pkg/pulsar/auth_polices.go new file mode 100644 index 0000000000..0200f11466 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/auth_polices.go @@ -0,0 +1,40 @@ +// 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 pulsar + +type AuthPolicies struct { + NamespaceAuth map[string]AuthAction `json:"namespace_auth"` + DestinationAuth map[string]map[string]AuthAction `json:"destination_auth"` + SubscriptionAuthRoles map[string][]string `json:"subscription_auth_roles"` +} + +func NewAuthPolicies() *AuthPolicies { + return &AuthPolicies{ + NamespaceAuth: make(map[string]AuthAction), + DestinationAuth: make(map[string]map[string]AuthAction), + SubscriptionAuthRoles: make(map[string][]string), + } +} + +type AuthAction string + +const ( + produce AuthAction = "produce" + consume AuthAction = "consume" + function AuthAction = "functions" +) diff --git a/pulsaradmin/pkg/pulsar/backlog_quota.go b/pulsaradmin/pkg/pulsar/backlog_quota.go new file mode 100644 index 0000000000..712e947a3b --- /dev/null +++ b/pulsaradmin/pkg/pulsar/backlog_quota.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 pulsar + +type BacklogQuota struct { + Limit int64 + Police RetentionPolicy + BacklogQuotaType BacklogQuotaType +} + +type RetentionPolicy int + +type BacklogQuotaType string + +const DestinationStorage BacklogQuotaType = "destination_storage" + +const ( + ProducerRequestHold RetentionPolicy = iota + ProducerException + ConsumerBacklogEviction +) + +func (rp RetentionPolicy) String() string { + names := [...]string{ + "ProducerRequestHold", + "ProducerException", + "ConsumerBacklogEviction", + } + + if rp < ProducerRequestHold || rp > ConsumerBacklogEviction { + return "Unknown Retention Policy" + } + + return names[rp] +} diff --git a/pulsaradmin/pkg/pulsar/bundles_data.go b/pulsaradmin/pkg/pulsar/bundles_data.go new file mode 100644 index 0000000000..5da16ae44e --- /dev/null +++ b/pulsaradmin/pkg/pulsar/bundles_data.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 pulsar + +type BundlesData struct { + Boundaries []string `json:"boundaries"` + NumBundles int `json:"numBundles"` +} + +func NewBundlesData(boundaries []string) BundlesData { + return BundlesData{ + Boundaries: boundaries, + NumBundles: len(boundaries) - 1, + } +} + +func NewBundlesDataWithNumBundles(numBundles int) *BundlesData { + return &BundlesData{ + Boundaries: nil, + NumBundles: numBundles, + } +} + +func NewDefaultBoundle() *BundlesData { + bundleData := NewBundlesDataWithNumBundles(1) + bundleData.Boundaries = append(bundleData.Boundaries, FirstBoundary, LastBoundary) + return bundleData +} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index ff0fe38f8d..c1453e812c 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -151,6 +151,11 @@ type PartitionedTopicMetadata struct { Partitions int `json:"partitions"` } +type NamespacesData struct { + NumBundles int `json:"numBundles"` + Clusters []string `json:"clusters"` +} + type SchemaData struct { Version int64 `json:"version"` Filename string `json:"filename"` @@ -162,9 +167,8 @@ type SchemaData struct { } type LookupData struct { - BrokerUrl string `json:"brokerUrl"` + BrokerUrl string `json:"brokerUrl"` BrokerUrlTls string `json:"brokerUrlTls"` - HttpUrl string `json:"httpUrl"` - HttpUrlTls string `json:"httpUrlTls"` + HttpUrl string `json:"httpUrl"` + HttpUrlTls string `json:"httpUrlTls"` } - diff --git a/pulsaradmin/pkg/pulsar/dispatch_rate.go b/pulsaradmin/pkg/pulsar/dispatch_rate.go new file mode 100644 index 0000000000..3813b03b27 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/dispatch_rate.go @@ -0,0 +1,44 @@ +// 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 pulsar + +type DispatchRate struct { + DispatchThrottlingRateInMsg int + DispatchThrottlingRateInByte int64 + RatePeriodInSecond int +} + +func NewDispatchRate() *DispatchRate { + return &DispatchRate{ + DispatchThrottlingRateInMsg: -1, + DispatchThrottlingRateInByte: -1, + RatePeriodInSecond: 1, + } +} + +type SubscribeRate struct { + SubscribeThrottlingRatePerConsumer int + RatePeriodInSecond int +} + +func NewSubscribeRate() *SubscribeRate { + return &SubscribeRate{ + SubscribeThrottlingRatePerConsumer: -1, + RatePeriodInSecond: 30, + } +} diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go new file mode 100644 index 0000000000..f6c9b432cd --- /dev/null +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -0,0 +1,140 @@ +// 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 pulsar + +type Namespaces interface { + // Get the list of all the namespaces for a certain tenant + GetNamespaces(tenant string) ([]string, error) + + // Get the list of all the topics under a certain namespace + GetTopics(namespace string) ([]string, error) + + // Get the dump all the policies specified for a namespace + GetPolicies(namespace string) (*Policies, error) + + // Creates a new empty namespace with no policies attached + CreateNamespace(namespace string) error + + // Creates a new empty namespace with no policies attached + CreateNsWithNumBundles(namespace string, numBundles int) error + + // Creates a new namespace with the specified policies + CreateNsWithPolices(namespace string, polices Policies) error + + // Creates a new empty namespace with no policies attached + CreateNsWithBundlesData(namespace string, bundleData *BundlesData) error + + // Delete an existing namespace + DeleteNamespace(namespace string) error + + // Delete an existing bundle in a namespace + DeleteNamespaceBundle(namespace string, bundleRange string) error +} + +type namespaces struct { + client *client + basePath string +} + +func (c *client) Namespaces() Namespaces { + return &namespaces{ + client: c, + basePath: "/namespaces", + } +} + +func (n *namespaces) GetNamespaces(tenant string) ([]string, error) { + var namespaces []string + endpoint := n.client.endpoint(n.basePath, tenant) + err := n.client.get(endpoint, &namespaces) + return namespaces, err +} + +func (n *namespaces) GetTopics(namespace string) ([]string, error) { + var topics []string + ns, err := GetNamespaceName(namespace) + if err != nil { + return nil, err + } + endpoint := n.client.endpoint(n.basePath, ns.String(), "topics") + err = n.client.get(endpoint, &topics) + return topics, err +} + +func (n *namespaces) GetPolicies(namespace string) (*Policies, error) { + var police Policies + ns, err := GetNamespaceName(namespace) + if err != nil { + return nil, err + } + endpoint := n.client.endpoint(n.basePath, ns.String()) + err = n.client.get(endpoint, &police) + return &police, err +} + +func (n *namespaces) CreateNsWithNumBundles(namespace string, numBundles int) error { + return n.CreateNsWithBundlesData(namespace, NewBundlesDataWithNumBundles(numBundles)) +} + +func (n *namespaces) CreateNsWithPolices(namespace string, policies Policies) error { + ns, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, ns.String()) + return n.client.put(endpoint, &policies, nil) +} + +func (n *namespaces) CreateNsWithBundlesData(namespace string, bundleData *BundlesData) error { + ns, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, ns.String()) + polices := new(Policies) + polices.Bundles = bundleData + + return n.client.put(endpoint, &polices, nil) +} + +func (n *namespaces) CreateNamespace(namespace string) error { + ns, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, ns.String()) + return n.client.put(endpoint, nil, nil) +} + +func (n *namespaces) DeleteNamespace(namespace string) error { + ns, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, ns.String()) + return n.client.delete(endpoint, nil) +} + +func (n *namespaces) DeleteNamespaceBundle(namespace string, bundleRange string) error { + ns, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, ns.String(), bundleRange) + return n.client.delete(endpoint, nil) +} diff --git a/pulsaradmin/pkg/pulsar/persistence_policies.go b/pulsaradmin/pkg/pulsar/persistence_policies.go new file mode 100644 index 0000000000..e6dfb7ddb5 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/persistence_policies.go @@ -0,0 +1,34 @@ +// 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 pulsar + +type PersistencePolicies struct { + BookkeeperEnsemble int + BookkeeperWriteQuorum int + BookkeeperAckQuorum int + ManagedLedgerMaxMarkDeleteRate float64 +} + +func NewPersistencePolicies() *PersistencePolicies { + return &PersistencePolicies{ + BookkeeperEnsemble: 2, + BookkeeperWriteQuorum: 2, + BookkeeperAckQuorum: 2, + ManagedLedgerMaxMarkDeleteRate: 0.0, + } +} diff --git a/pulsaradmin/pkg/pulsar/policies.go b/pulsaradmin/pkg/pulsar/policies.go new file mode 100644 index 0000000000..d8336db522 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/policies.go @@ -0,0 +1,82 @@ +// 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 pulsar + +const ( + FirstBoundary string = "0x00000000" + LastBoundary string = "0xffffffff" +) + +type Policies struct { + AuthPolicies AuthPolicies `json:"auth_policies"` + ReplicationClusters []string `json:"replication_clusters"` + Bundles *BundlesData `json:"bundles"` + BacklogQuotaMap map[BacklogQuotaType]BacklogQuota `json:"backlog_quota_map"` + TopicDispatchRate map[string]DispatchRate `json:"topicDispatchRate"` + SubscriptionDispatchRate map[string]DispatchRate `json:"subscriptionDispatchRate"` + ReplicatorDispatchRate map[string]DispatchRate `json:"replicatorDispatchRate"` + ClusterSubscribeRate map[string]SubscribeRate `json:"clusterSubscribeRate"` + Persistence *PersistencePolicies `json:"persistence"` + DeduplicationEnabled bool `json:"deduplicationEnabled"` + LatencyStatsSampleRate map[string]int `json:"latency_stats_sample_rate"` + MessageTtlInSeconds int `json:"message_ttl_in_seconds"` + RetentionPolicies *RetentionPolicies `json:"retention_policies"` + Deleted bool `json:"deleted"` + AntiAffinityGroup string `json:"antiAffinityGroup"` + EncryptionRequired bool `json:"encryption_required"` + SubscriptionAuthMode SubscriptionAuthMode `json:"subscription_auth_mode"` + MaxProducersPerTopic int `json:"max_producers_per_topic"` + MaxConsumersPerTopic int `json:"max_consumers_per_topic"` + MaxConsumersPerSubscription int `json:"max_consumers_per_subscription"` + CompactionThreshold int64 `json:"compaction_threshold"` + OffloadThreshold int64 `json:"offload_threshold"` + OffloadDeletionLagMs int64 `json:"offload_deletion_lag_ms"` + SchemaAutoUpdateCompatibilityStrategy SchemaAutoUpdateCompatibilityStrategy `json:"schema_auto_update_compatibility_strategy"` + SchemaValidationEnforced bool `json:"schema_validation_enforced"` +} + +func NewDefaultPolicies() *Policies { + return &Policies{ + AuthPolicies: *NewAuthPolicies(), + ReplicationClusters: make([]string, 0, 10), + BacklogQuotaMap: make(map[BacklogQuotaType]BacklogQuota), + TopicDispatchRate: make(map[string]DispatchRate), + SubscriptionDispatchRate: make(map[string]DispatchRate), + ReplicatorDispatchRate: make(map[string]DispatchRate), + ClusterSubscribeRate: make(map[string]SubscribeRate), + LatencyStatsSampleRate: make(map[string]int), + MessageTtlInSeconds: 0, + Deleted: false, + EncryptionRequired: false, + SubscriptionAuthMode: None, + MaxProducersPerTopic: 0, + MaxConsumersPerSubscription: 0, + MaxConsumersPerTopic: 0, + CompactionThreshold: 0, + OffloadThreshold: -1, + SchemaAutoUpdateCompatibilityStrategy: Full, + SchemaValidationEnforced: false, + } +} + +type SubscriptionAuthMode string + +const ( + None SubscriptionAuthMode = "None" + Prefix SubscriptionAuthMode = "Prefix" +) diff --git a/pulsaradmin/pkg/pulsar/retention_policies.go b/pulsaradmin/pkg/pulsar/retention_policies.go new file mode 100644 index 0000000000..c25214c083 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/retention_policies.go @@ -0,0 +1,23 @@ +// 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 pulsar + +type RetentionPolicies struct { + RetentionTimeInMinutes int + RetentionSizeInMB int64 +} diff --git a/pulsaradmin/pkg/pulsar/schema_strategy.go b/pulsaradmin/pkg/pulsar/schema_strategy.go new file mode 100644 index 0000000000..66b4752595 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/schema_strategy.go @@ -0,0 +1,31 @@ +// 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 pulsar + +type SchemaAutoUpdateCompatibilityStrategy string + +const ( + AutoUpdateDisabled SchemaAutoUpdateCompatibilityStrategy = "AutoUpdateDisabled" + Backward SchemaAutoUpdateCompatibilityStrategy = "Backward" + Forward SchemaAutoUpdateCompatibilityStrategy = "Forward" + Full SchemaAutoUpdateCompatibilityStrategy = "Full" + AlwaysCompatible SchemaAutoUpdateCompatibilityStrategy = "AlwaysCompatible" + BackwardTransitive SchemaAutoUpdateCompatibilityStrategy = "BackwardTransitive" + ForwardTransitive SchemaAutoUpdateCompatibilityStrategy = "ForwardTransitive" + FullTransitive SchemaAutoUpdateCompatibilityStrategy = "FullTransitive" +) From 24d7160f8b09b2a950ade73850069d253f22f89c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Tue, 17 Sep 2019 11:05:08 +0800 Subject: [PATCH 115/348] Add backlog, retention and ttl commands for pulsarctl (streamnative/pulsarctl#61) * Add backlog, retention and ttl commands for pulsarctl Signed-off-by: xiaolong.ran --- pulsaradmin/pkg/pulsar/backlog_quota.go | 34 +++---- pulsaradmin/pkg/pulsar/data.go | 8 +- pulsaradmin/pkg/pulsar/namespace.go | 94 ++++++++++++++++++++ pulsaradmin/pkg/pulsar/retention_policies.go | 11 ++- 4 files changed, 122 insertions(+), 25 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/backlog_quota.go b/pulsaradmin/pkg/pulsar/backlog_quota.go index 712e947a3b..8d83b78717 100644 --- a/pulsaradmin/pkg/pulsar/backlog_quota.go +++ b/pulsaradmin/pkg/pulsar/backlog_quota.go @@ -18,33 +18,25 @@ package pulsar type BacklogQuota struct { - Limit int64 - Police RetentionPolicy - BacklogQuotaType BacklogQuotaType + Limit int64 `json:"limit"` + Policy RetentionPolicy `json:"policy"` } -type RetentionPolicy int +func NewBacklogQuota(limit int64, policy RetentionPolicy) BacklogQuota { + return BacklogQuota{ + Limit: limit, + Policy: policy, + } +} + +type RetentionPolicy string type BacklogQuotaType string const DestinationStorage BacklogQuotaType = "destination_storage" const ( - ProducerRequestHold RetentionPolicy = iota - ProducerException - ConsumerBacklogEviction + ProducerRequestHold RetentionPolicy = "producer_request_hold" + ProducerException RetentionPolicy = "producer_exception" + ConsumerBacklogEviction RetentionPolicy = "consumer_backlog_eviction" ) - -func (rp RetentionPolicy) String() string { - names := [...]string{ - "ProducerRequestHold", - "ProducerException", - "ConsumerBacklogEviction", - } - - if rp < ProducerRequestHold || rp > ConsumerBacklogEviction { - return "Unknown Retention Policy" - } - - return names[rp] -} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index c1453e812c..083f88b71b 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -152,8 +152,12 @@ type PartitionedTopicMetadata struct { } type NamespacesData struct { - NumBundles int `json:"numBundles"` - Clusters []string `json:"clusters"` + NumBundles int `json:"numBundles"` + Clusters []string `json:"clusters"` + MessageTTL int `json:"messageTTL"` + RetentionTimeStr string `json:"retentionTimeStr"` + LimitStr string `json:"limitStr"` + PolicyStr string `json:"policyStr"` } type SchemaData struct { diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index f6c9b432cd..15f10d93be 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -44,6 +44,27 @@ type Namespaces interface { // Delete an existing bundle in a namespace DeleteNamespaceBundle(namespace string, bundleRange string) error + + // Set the messages Time to Live for all the topics within a namespace + SetNamespaceMessageTTL(namespace string, ttlInSeconds int) error + + // Get the message TTL for a namespace + GetNamespaceMessageTTL(namespace string) (int, error) + + // Get the retention configuration for a namespace + GetRetention(namespace string) (*RetentionPolicies, error) + + // Set the retention configuration for all the topics on a namespace + SetRetention(namespace string, policy RetentionPolicies) error + + // Get backlog quota map on a namespace + GetBacklogQuotaMap(namespace string) (map[BacklogQuotaType]BacklogQuota, error) + + // Set a backlog quota for all the topics on a namespace + SetBacklogQuota(namespace string, backlogQuota BacklogQuota) error + + // Remove a backlog quota policy from a namespace + RemoveBacklogQuota(namespace string) error } type namespaces struct { @@ -138,3 +159,76 @@ func (n *namespaces) DeleteNamespaceBundle(namespace string, bundleRange string) endpoint := n.client.endpoint(n.basePath, ns.String(), bundleRange) return n.client.delete(endpoint, nil) } + +func (n *namespaces) GetNamespaceMessageTTL(namespace string) (int, error) { + var ttl int + nsName, err := GetNamespaceName(namespace) + if err != nil { + return 0, err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "messageTTL") + err = n.client.get(endpoint, &ttl) + return ttl, err +} + +func (n *namespaces) SetNamespaceMessageTTL(namespace string, ttlInSeconds int) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + + endpoint := n.client.endpoint(n.basePath, nsName.String(), "messageTTL") + return n.client.post(endpoint, &ttlInSeconds, nil) +} + +func (n *namespaces) SetRetention(namespace string, policy RetentionPolicies) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "retention") + return n.client.post(endpoint, &policy, nil) +} + +func (n *namespaces) GetRetention(namespace string) (*RetentionPolicies, error) { + var policy RetentionPolicies + nsName, err := GetNamespaceName(namespace) + if err != nil { + return nil, err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "retention") + err = n.client.get(endpoint, &policy) + return &policy, err +} + +func (n *namespaces) GetBacklogQuotaMap(namespace string) (map[BacklogQuotaType]BacklogQuota, error) { + var backlogQuotaMap map[BacklogQuotaType]BacklogQuota + nsName, err := GetNamespaceName(namespace) + if err != nil { + return nil, err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuotaMap") + err = n.client.get(endpoint, &backlogQuotaMap) + return backlogQuotaMap, err +} + +func (n *namespaces) SetBacklogQuota(namespace string, backlogQuota BacklogQuota) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuota") + return n.client.post(endpoint, &backlogQuota, nil) +} + +func (n *namespaces) RemoveBacklogQuota(namespace string) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuota") + params := map[string]string{ + "backlogQuotaType": string(DestinationStorage), + } + return n.client.deleteWithQueryParams(endpoint, nil, params) +} diff --git a/pulsaradmin/pkg/pulsar/retention_policies.go b/pulsaradmin/pkg/pulsar/retention_policies.go index c25214c083..23e3d7c736 100644 --- a/pulsaradmin/pkg/pulsar/retention_policies.go +++ b/pulsaradmin/pkg/pulsar/retention_policies.go @@ -18,6 +18,13 @@ package pulsar type RetentionPolicies struct { - RetentionTimeInMinutes int - RetentionSizeInMB int64 + RetentionTimeInMinutes int `json:"retentionTimeInMinutes"` + RetentionSizeInMB int64 `json:"retentionSizeInMB"` +} + +func NewRetentionPolicies(retentionTimeInMinutes int, retentionSizeInMB int) RetentionPolicies { + return RetentionPolicies{ + RetentionTimeInMinutes: retentionTimeInMinutes, + RetentionSizeInMB: int64(retentionSizeInMB), + } } From 20dbbd2d2904cc6063d3dc41fcceb940469aec88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Tue, 17 Sep 2019 19:24:19 +0800 Subject: [PATCH 116/348] Revert "Add backlog, retention and ttl commands for pulsarctl (streamnative/pulsarctl#61)" (#62) This reverts commit c844df17c3857818458e5b90d1b6afb1c5b9a560. --- pulsaradmin/pkg/pulsar/backlog_quota.go | 34 ++++--- pulsaradmin/pkg/pulsar/data.go | 8 +- pulsaradmin/pkg/pulsar/namespace.go | 94 -------------------- pulsaradmin/pkg/pulsar/retention_policies.go | 11 +-- 4 files changed, 25 insertions(+), 122 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/backlog_quota.go b/pulsaradmin/pkg/pulsar/backlog_quota.go index 8d83b78717..712e947a3b 100644 --- a/pulsaradmin/pkg/pulsar/backlog_quota.go +++ b/pulsaradmin/pkg/pulsar/backlog_quota.go @@ -18,25 +18,33 @@ package pulsar type BacklogQuota struct { - Limit int64 `json:"limit"` - Policy RetentionPolicy `json:"policy"` + Limit int64 + Police RetentionPolicy + BacklogQuotaType BacklogQuotaType } -func NewBacklogQuota(limit int64, policy RetentionPolicy) BacklogQuota { - return BacklogQuota{ - Limit: limit, - Policy: policy, - } -} - -type RetentionPolicy string +type RetentionPolicy int type BacklogQuotaType string const DestinationStorage BacklogQuotaType = "destination_storage" const ( - ProducerRequestHold RetentionPolicy = "producer_request_hold" - ProducerException RetentionPolicy = "producer_exception" - ConsumerBacklogEviction RetentionPolicy = "consumer_backlog_eviction" + ProducerRequestHold RetentionPolicy = iota + ProducerException + ConsumerBacklogEviction ) + +func (rp RetentionPolicy) String() string { + names := [...]string{ + "ProducerRequestHold", + "ProducerException", + "ConsumerBacklogEviction", + } + + if rp < ProducerRequestHold || rp > ConsumerBacklogEviction { + return "Unknown Retention Policy" + } + + return names[rp] +} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index 083f88b71b..c1453e812c 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -152,12 +152,8 @@ type PartitionedTopicMetadata struct { } type NamespacesData struct { - NumBundles int `json:"numBundles"` - Clusters []string `json:"clusters"` - MessageTTL int `json:"messageTTL"` - RetentionTimeStr string `json:"retentionTimeStr"` - LimitStr string `json:"limitStr"` - PolicyStr string `json:"policyStr"` + NumBundles int `json:"numBundles"` + Clusters []string `json:"clusters"` } type SchemaData struct { diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index 15f10d93be..f6c9b432cd 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -44,27 +44,6 @@ type Namespaces interface { // Delete an existing bundle in a namespace DeleteNamespaceBundle(namespace string, bundleRange string) error - - // Set the messages Time to Live for all the topics within a namespace - SetNamespaceMessageTTL(namespace string, ttlInSeconds int) error - - // Get the message TTL for a namespace - GetNamespaceMessageTTL(namespace string) (int, error) - - // Get the retention configuration for a namespace - GetRetention(namespace string) (*RetentionPolicies, error) - - // Set the retention configuration for all the topics on a namespace - SetRetention(namespace string, policy RetentionPolicies) error - - // Get backlog quota map on a namespace - GetBacklogQuotaMap(namespace string) (map[BacklogQuotaType]BacklogQuota, error) - - // Set a backlog quota for all the topics on a namespace - SetBacklogQuota(namespace string, backlogQuota BacklogQuota) error - - // Remove a backlog quota policy from a namespace - RemoveBacklogQuota(namespace string) error } type namespaces struct { @@ -159,76 +138,3 @@ func (n *namespaces) DeleteNamespaceBundle(namespace string, bundleRange string) endpoint := n.client.endpoint(n.basePath, ns.String(), bundleRange) return n.client.delete(endpoint, nil) } - -func (n *namespaces) GetNamespaceMessageTTL(namespace string) (int, error) { - var ttl int - nsName, err := GetNamespaceName(namespace) - if err != nil { - return 0, err - } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "messageTTL") - err = n.client.get(endpoint, &ttl) - return ttl, err -} - -func (n *namespaces) SetNamespaceMessageTTL(namespace string, ttlInSeconds int) error { - nsName, err := GetNamespaceName(namespace) - if err != nil { - return err - } - - endpoint := n.client.endpoint(n.basePath, nsName.String(), "messageTTL") - return n.client.post(endpoint, &ttlInSeconds, nil) -} - -func (n *namespaces) SetRetention(namespace string, policy RetentionPolicies) error { - nsName, err := GetNamespaceName(namespace) - if err != nil { - return err - } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "retention") - return n.client.post(endpoint, &policy, nil) -} - -func (n *namespaces) GetRetention(namespace string) (*RetentionPolicies, error) { - var policy RetentionPolicies - nsName, err := GetNamespaceName(namespace) - if err != nil { - return nil, err - } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "retention") - err = n.client.get(endpoint, &policy) - return &policy, err -} - -func (n *namespaces) GetBacklogQuotaMap(namespace string) (map[BacklogQuotaType]BacklogQuota, error) { - var backlogQuotaMap map[BacklogQuotaType]BacklogQuota - nsName, err := GetNamespaceName(namespace) - if err != nil { - return nil, err - } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuotaMap") - err = n.client.get(endpoint, &backlogQuotaMap) - return backlogQuotaMap, err -} - -func (n *namespaces) SetBacklogQuota(namespace string, backlogQuota BacklogQuota) error { - nsName, err := GetNamespaceName(namespace) - if err != nil { - return err - } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuota") - return n.client.post(endpoint, &backlogQuota, nil) -} - -func (n *namespaces) RemoveBacklogQuota(namespace string) error { - nsName, err := GetNamespaceName(namespace) - if err != nil { - return err - } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuota") - params := map[string]string{ - "backlogQuotaType": string(DestinationStorage), - } - return n.client.deleteWithQueryParams(endpoint, nil, params) -} diff --git a/pulsaradmin/pkg/pulsar/retention_policies.go b/pulsaradmin/pkg/pulsar/retention_policies.go index 23e3d7c736..c25214c083 100644 --- a/pulsaradmin/pkg/pulsar/retention_policies.go +++ b/pulsaradmin/pkg/pulsar/retention_policies.go @@ -18,13 +18,6 @@ package pulsar type RetentionPolicies struct { - RetentionTimeInMinutes int `json:"retentionTimeInMinutes"` - RetentionSizeInMB int64 `json:"retentionSizeInMB"` -} - -func NewRetentionPolicies(retentionTimeInMinutes int, retentionSizeInMB int) RetentionPolicies { - return RetentionPolicies{ - RetentionTimeInMinutes: retentionTimeInMinutes, - RetentionSizeInMB: int64(retentionSizeInMB), - } + RetentionTimeInMinutes int + RetentionSizeInMB int64 } From 416e8c8dd677d77c4ba3fca4e3283910c1fa6579 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 17 Sep 2019 20:56:24 +0800 Subject: [PATCH 117/348] Add topic stats command (streamnative/pulsarctl#46) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add command topic `stats`, `internal-stats`, `partition-stats` * topic stats * ``` ➜ pulsarctl git:(topic-stats) ✗ ./pulsarctl topic stats -h USED FOR: This command is used for getting the stats for an existing topic and its connected producers and consumers. (All the rates are computed over a 1 minute window and are relative the last completed 1 minute period) REQUIRED PERMISSION: This command requires namespace admin permissions. EXAMPLES: #Get the non-partitioned topic stats pulsarctl topic stats #Get the partitioned topic stats pulsarctl topic stats --partition #Get the partitioned topic stats and per partition stats pulsarctl topic stats --partition --per-partition OUTPUT: #Get the non-partitioned topic stats { "msgRateIn": 0, "msgRateOut": 0, "msgThroughputIn": 0, "msgThroughputOut": 0, "averageMsgSize": 0, "storageSize": 0, "publishers": [], "subscriptions": {}, "replication": {}, "deduplicationStatus": "Disabled" } #Get the partitioned topic stats { "msgRateIn": 0, "msgRateOut": 0, "msgThroughputIn": 0, "msgThroughputOut": 0, "averageMsgSize": 0, "storageSize": 0, "publishers": [], "subscriptions": {}, "replication": {}, "deduplicationStatus": "", "metadata": { "partitions": 1 }, "partitions": {} } #Get the partitioned topic stats and per partition topic stats { "msgRateIn": 0, "msgRateOut": 0, "msgThroughputIn": 0, "msgThroughputOut": 0, "averageMsgSize": 0, "storageSize": 0, "publishers": [], "subscriptions": {}, "replication": {}, "deduplicationStatus": "", "metadata": { "partitions": 1 }, "partitions": { "": { "msgRateIn": 0, "msgRateOut": 0, "msgThroughputIn": 0, "msgThroughputOut": 0, "averageMsgSize": 0, "storageSize": 0, "publishers": [], "subscriptions": {}, "replication": {}, "deduplicationStatus": "" } } } #the topic name is not specified [✖] only one argument is allowed to be used as a name #the specified topic is not exist or the specified topic is a partitioned-topic and you don't specified --partition or the specified topic is a non-partitioned topic and you specified --partition code: 404 reason: Topic not found #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics stats [flags] ``` * topic internal-stats * ``` ➜ pulsarctl-yong git:(topic-stats) ./pulsarctl topic internal-stats -h USED FOR: This command is used for getting the internal stats for an existing non-partitioned topic. REQUIRED PERMISSION: This command requires namespace admin permissions. EXAMPLES: #Get internal stats for an existing non-partitioned-topic pulsarctl topic internal-stats OUTPUT: #normal output { "entriesAddedCounter": 0, "numberOfEntries": 0, "totalSize": 0, "currentLedgerEntries": 0, "currentLedgerSize": 0, "lastLedgerCreatedTimestamp": "", "lastLedgerCreationFailureTimestamp": "", "waitingCursorsCount": 0, "pendingAddEntriesCount": 0, "lastConfirmedEntry": "", "state": "", "ledgers": [ { "ledgerId": 0, "entries": 0, "size": 0, "offloaded": false } ], "cursors": {} } #the topic name is not specified [✖] only one argument is allowed to be used as a name #the specified topic is not exist or the specified topic is a partitioned topic [✖] code: 404 reason: Topic not found #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics internal-stats [flags] Aliases: internal-stats, ``` --- pulsaradmin/pkg/pulsar/admin.go | 13 +- pulsaradmin/pkg/pulsar/data.go | 116 ++++++++++++++++++ pulsaradmin/pkg/pulsar/namespace_name_test.go | 2 +- pulsaradmin/pkg/pulsar/topic.go | 29 ++++- 4 files changed, 156 insertions(+), 4 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index c4ea1c533b..c2d75686f3 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -146,12 +146,20 @@ func (c *client) endpoint(componentPath string, parts ...string) string { // get is used to do a GET request against an endpoint // and deserialize the response into an interface -func (c *client) getAndDecode(endpoint string, obj interface{}, decode bool) ([]byte, error) { +func (c *client) getWithQueryParams(endpoint string, obj interface{}, params map[string]string, decode bool) ([]byte, error) { req, err := c.newRequest(http.MethodGet, endpoint) if err != nil { return nil, err } + if params != nil { + query := req.url.Query() + for k, v := range params { + query.Add(k, v) + } + req.params = query + } + resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return nil, err @@ -174,7 +182,7 @@ func (c *client) getAndDecode(endpoint string, obj interface{}, decode bool) ([] } func (c *client) get(endpoint string, obj interface{}) error { - _, err := c.getAndDecode(endpoint, obj, true) + _, err := c.getWithQueryParams(endpoint, obj, nil, true) return err } @@ -215,6 +223,7 @@ func (c *client) deleteWithQueryParams(endpoint string, obj interface{}, params for k, v := range params { query.Add(k, v) } + req.params = query } resp, err := checkSuccessful(c.doRequest(req)) diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index c1453e812c..633e5b8b82 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -156,6 +156,122 @@ type NamespacesData struct { Clusters []string `json:"clusters"` } +type TopicStats struct { + MsgRateIn float64 `json:"msgRateIn"` + MsgRateOut float64 `json:"msgRateOut"` + MsgThroughputIn float64 `json:"msgThroughputIn"` + MsgThroughputOut float64 `json:"msgThroughputOut"` + AverageMsgSize float64 `json:"averageMsgSize"` + StorageSize int64 `json:"storageSize"` + Publishers []PublisherStats `json:"publishers"` + Subscriptions map[string]SubscriptionStats `json:"subscriptions"` + Replication map[string]ReplicatorStats `json:"replication"` + DeDuplicationStatus string `json:"deduplicationStatus"` +} + +type PublisherStats struct { + ProducerId int64 `json:"producerId"` + MsgRateIn float64 `json:"msgRateIn"` + MsgThroughputIn float64 `json:"msgThroughputIn"` + AverageMsgSize float64 `json:"averageMsgSize"` + Metadata map[string]string `json:"metadata"` +} + +type SubscriptionStats struct { + MsgRateOut float64 `json:"msgRateOut"` + MsgThroughputOut float64 `json:"msgThroughputOut"` + MsgRateRedeliver float64 `json:"msgRateRedeliver"` + MsgRateExpired float64 `json:"msgRateExpired"` + MsgBacklog int64 `json:"msgBacklog"` + BlockedSubscriptionOnUnackedMsgs bool `json:"blockedSubscriptionOnUnackedMsgs"` + MsgDelayed int64 `json:"msgDelayed"` + unAckedMessages int64 `json:"unackedMessages"` + SubType string `json:"type"` + ActiveConsumerName string `json:"activeConsumerName"` + Consumers []ConsumerStats `json:"consumers"` + IsReplicated bool `json:"isReplicated"` +} + +type ConsumerStats struct { + ConsumerName string `json:"consumerName"` + MsgRateOut float64 `json:"msgRateOut"` + MsgThroughputOut float64 `json:"msgThroughputOut"` + MsgRateRedeliver float64 `json:"msgRateRedeliver"` + AvailablePermits int `json:"availablePermits"` + UnAckedMessages int `json:"unackedMessages"` + BlockedConsumerOnUnAckedMsgs bool `json:"blockedConsumerOnUnackedMsgs"` + Metadata map[string]string `json:"metadata"` +} + +type ReplicatorStats struct { + MsgRateIn float64 `json:"msgRateIn"` + MsgRateOut float64 `json:"msgRateOut"` + MsgThroughputIn float64 `json:"msgThroughputIn"` + MsgThroughputOut float64 `json:"msgThroughputOut"` + MsgRateExpired float64 `json:"msgRateExpired"` + ReplicationBacklog int64 `json:"replicationBacklog"` + Connected bool `json:"connected"` + ReplicationDelayInSeconds int64 `json:"replicationDelayInSeconds"` + InboundConnection string `json:"inboundConnection"` + InboundConnectedSince string `json:"inboundConnectedSince"` + OutboundConnection string `json:"outboundConnection"` + OutboundConnectedSince string `json:"outboundConnectedSince"` +} + +type PersistentTopicInternalStats struct { + EntriesAddedCounter int64 `json:"entriesAddedCounter"` + NumberOfEntries int64 `json:"numberOfEntries"` + TotalSize int64 `json:"totalSize"` + CurrentLedgerEntries int64 `json:"currentLedgerEntries"` + CurrentLedgerSize int64 `json:"currentLedgerSize"` + LastLedgerCreatedTimestamp string `json:"lastLedgerCreatedTimestamp"` + LastLedgerCreationFailureTimestamp string `json:"lastLedgerCreationFailureTimestamp"` + WaitingCursorsCount int `json:"waitingCursorsCount"` + PendingAddEntriesCount int `json:"pendingAddEntriesCount"` + LastConfirmedEntry string `json:"lastConfirmedEntry"` + State string `json:"state"` + Ledgers []LedgerInfo `json:"ledgers"` + Cursors map[string]CursorStats `json:"cursors"` +} + +type LedgerInfo struct { + LedgerId int64 `json:"ledgerId"` + Entries int64 `json:"entries"` + Size int64 `json:"size"` + Offloaded bool `json:"offloaded"` +} + +type CursorStats struct { + MarkDeletePosition string `json:"markDeletePosition"` + ReadPosition string `json:"readPosition"` + WaitingReadOp bool `json:"waitingReadOp"` + PendingReadOps int `json:"pendingReadOps"` + MessagesConsumedCounter int64 `json:"messagesConsumedCounter"` + CursorLedger int64 `json:"cursorLedger"` + CursorLedgerLastEntry int64 `json:"cursorLedgerLastEntry"` + IndividuallyDeletedMessages string `json:"individuallyDeletedMessages"` + LastLedgerWitchTimestamp string `json:"lastLedgerWitchTimestamp"` + State string `json:"state"` + NumberOfEntriesSinceFirstNotAckedMessage int64 `json:"numberOfEntriesSinceFirstNotAckedMessage"` + TotalNonContiguousDeletedMessagesRange int `json:"totalNonContiguousDeletedMessagesRange"` + Properties map[string]int64 `json:"properties"` +} + +type PartitionedTopicStats struct { + MsgRateIn float64 `json:"msgRateIn"` + MsgRateOut float64 `json:"msgRateOut"` + MsgThroughputIn float64 `json:"msgThroughputIn"` + MsgThroughputOut float64 `json:"msgThroughputOut"` + AverageMsgSize float64 `json:"averageMsgSize"` + StorageSize int64 `json:"storageSize"` + Publishers []PublisherStats `json:"publishers"` + Subscriptions map[string]SubscriptionStats `json:"subscriptions"` + Replication map[string]ReplicatorStats `json:"replication"` + DeDuplicationStatus string `json:"deduplicationStatus"` + Metadata PartitionedTopicMetadata `json:"metadata"` + Partitions map[string]TopicStats `json:"partitions"` +} + type SchemaData struct { Version int64 `json:"version"` Filename string `json:"filename"` diff --git a/pulsaradmin/pkg/pulsar/namespace_name_test.go b/pulsaradmin/pkg/pulsar/namespace_name_test.go index 8d1210cf62..b1c7c2c8ef 100644 --- a/pulsaradmin/pkg/pulsar/namespace_name_test.go +++ b/pulsaradmin/pkg/pulsar/namespace_name_test.go @@ -1,4 +1,4 @@ -package pulsar +package pulsar import ( "github.com/stretchr/testify/assert" diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 12377ebfc7..d193e6ed2a 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -14,6 +14,9 @@ type Topics interface { Lookup(TopicName) (LookupData, error) GetBundleRange(TopicName) (string, error) GetLastMessageId(TopicName) (MessageId, error) + GetStats(TopicName) (TopicStats, error) + GetInternalStats(TopicName) (PersistentTopicInternalStats, error) + GetPartitionedStats(TopicName, bool) (PartitionedTopicStats, error) } type topics struct { @@ -118,7 +121,7 @@ func (t *topics) Lookup(topic TopicName) (LookupData, error) { func (t *topics) GetBundleRange(topic TopicName) (string, error) { endpoint := fmt.Sprintf("%s/%s/%s", t.lookupPath, topic.GetRestPath(), "bundle") - data, err := t.client.getAndDecode(endpoint, nil, false) + data, err := t.client.getWithQueryParams(endpoint, nil, nil, false) return string(data), err } @@ -128,3 +131,27 @@ func (t *topics) GetLastMessageId(topic TopicName) (MessageId, error) { err := t.client.get(endpoint, &messageId) return messageId, err } + +func (t *topics) GetStats(topic TopicName) (TopicStats, error) { + var stats TopicStats + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "stats") + err := t.client.get(endpoint, &stats) + return stats, err +} + +func (t *topics) GetInternalStats(topic TopicName) (PersistentTopicInternalStats, error) { + var stats PersistentTopicInternalStats + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "internalStats") + err := t.client.get(endpoint, &stats) + return stats, err +} + +func (t *topics) GetPartitionedStats(topic TopicName, perPartition bool) (PartitionedTopicStats, error) { + var stats PartitionedTopicStats + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitioned-stats") + params := map[string]string{ + "perPartition": strconv.FormatBool(perPartition), + } + _, err := t.client.getWithQueryParams(endpoint, &stats, params, true) + return stats, err +} From 04bfc275473877f92b3206596f1e175616fdfb31 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Wed, 18 Sep 2019 23:45:17 +0800 Subject: [PATCH 118/348] Add antiAffinityGroup, persistence, clusters, unload, spliteBundle and deduplication for pulsartl (streamnative/pulsarctl#66) Add some commands for pulsartl, as follows: - getAntiAffinityGroup - setAntiAffinityGroup - deleteAntiAffinityGroup - getAntiAffinityNamespaces - getPersistence - setPersistence - setDeduplication - setReplicationClusters - getReplicationClusters - unload - splitBundle --- pulsaradmin/pkg/pulsar/admin.go | 14 + pulsaradmin/pkg/pulsar/api_version.go | 6 +- pulsaradmin/pkg/pulsar/api_version_test.go | 2 +- pulsaradmin/pkg/pulsar/backlog_quota.go | 34 +- pulsaradmin/pkg/pulsar/data.go | 19 +- pulsaradmin/pkg/pulsar/namespace.go | 291 ++++++++++++++++++ .../pkg/pulsar/persistence_policies.go | 31 +- pulsaradmin/pkg/pulsar/retention_policies.go | 11 +- 8 files changed, 374 insertions(+), 34 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index c2d75686f3..f3508d98fc 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -146,7 +146,9 @@ func (c *client) endpoint(componentPath string, parts ...string) string { // get is used to do a GET request against an endpoint // and deserialize the response into an interface + func (c *client) getWithQueryParams(endpoint string, obj interface{}, params map[string]string, decode bool) ([]byte, error) { + req, err := c.newRequest(http.MethodGet, endpoint) if err != nil { return nil, err @@ -187,12 +189,24 @@ func (c *client) get(endpoint string, obj interface{}) error { } func (c *client) put(endpoint string, in, obj interface{}) error { + return c.putWithQueryParams(endpoint, in, obj, nil) +} + +func (c *client) putWithQueryParams(endpoint string, in, obj interface{}, params map[string]string) error { req, err := c.newRequest(http.MethodPut, endpoint) if err != nil { return err } req.obj = in + if params != nil { + query := req.url.Query() + for k, v := range params { + query.Add(k, v) + } + req.params = query + } + resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return err diff --git a/pulsaradmin/pkg/pulsar/api_version.go b/pulsaradmin/pkg/pulsar/api_version.go index ba055ef191..fd239798c7 100644 --- a/pulsaradmin/pkg/pulsar/api_version.go +++ b/pulsaradmin/pkg/pulsar/api_version.go @@ -1,18 +1,19 @@ package pulsar type ApiVersion int + const ( V1 ApiVersion = iota V2 V3 ) -const DefaultApiVersion = "v2" +const DefaultApiVersion = "v2" func (v ApiVersion) String() string { switch v { case V1: - return "v1" + return "" case V2: return "v2" case V3: @@ -21,4 +22,3 @@ func (v ApiVersion) String() string { return DefaultApiVersion } - diff --git a/pulsaradmin/pkg/pulsar/api_version_test.go b/pulsaradmin/pkg/pulsar/api_version_test.go index d9982661ac..8de8fac7ef 100644 --- a/pulsaradmin/pkg/pulsar/api_version_test.go +++ b/pulsaradmin/pkg/pulsar/api_version_test.go @@ -6,7 +6,7 @@ import ( ) func TestApiVersion_String(t *testing.T) { - assert.Equal(t, "v1", V1.String()) + assert.Equal(t, "", V1.String()) assert.Equal(t, "v2", V2.String()) assert.Equal(t, "v3", V3.String()) } \ No newline at end of file diff --git a/pulsaradmin/pkg/pulsar/backlog_quota.go b/pulsaradmin/pkg/pulsar/backlog_quota.go index 712e947a3b..8d83b78717 100644 --- a/pulsaradmin/pkg/pulsar/backlog_quota.go +++ b/pulsaradmin/pkg/pulsar/backlog_quota.go @@ -18,33 +18,25 @@ package pulsar type BacklogQuota struct { - Limit int64 - Police RetentionPolicy - BacklogQuotaType BacklogQuotaType + Limit int64 `json:"limit"` + Policy RetentionPolicy `json:"policy"` } -type RetentionPolicy int +func NewBacklogQuota(limit int64, policy RetentionPolicy) BacklogQuota { + return BacklogQuota{ + Limit: limit, + Policy: policy, + } +} + +type RetentionPolicy string type BacklogQuotaType string const DestinationStorage BacklogQuotaType = "destination_storage" const ( - ProducerRequestHold RetentionPolicy = iota - ProducerException - ConsumerBacklogEviction + ProducerRequestHold RetentionPolicy = "producer_request_hold" + ProducerException RetentionPolicy = "producer_exception" + ConsumerBacklogEviction RetentionPolicy = "consumer_backlog_eviction" ) - -func (rp RetentionPolicy) String() string { - names := [...]string{ - "ProducerRequestHold", - "ProducerException", - "ConsumerBacklogEviction", - } - - if rp < ProducerRequestHold || rp > ConsumerBacklogEviction { - return "Unknown Retention Policy" - } - - return names[rp] -} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index 633e5b8b82..d9370a33b7 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -152,8 +152,23 @@ type PartitionedTopicMetadata struct { } type NamespacesData struct { - NumBundles int `json:"numBundles"` - Clusters []string `json:"clusters"` + NumBundles int `json:"numBundles"` + Clusters []string `json:"clusters"` + ClusterIds string `json:"clusterIds"` + MessageTTL int `json:"messageTTL"` + RetentionTimeStr string `json:"retentionTimeStr"` + LimitStr string `json:"limitStr"` + PolicyStr string `json:"policyStr"` + AntiAffinityGroup string `json:"antiAffinityGroup"` + Tenant string `json:"tenant"` + Cluster string `json:"cluster"` + BookkeeperEnsemble int `json:"bookkeeperEnsemble"` + BookkeeperWriteQuorum int `json:"bookkeeperWriteQuorum"` + BookkeeperAckQuorum int `json:"bookkeeperAckQuorum"` + ManagedLedgerMaxMarkDeleteRate float64 `json:"managedLedgerMaxMarkDeleteRate"` + Enable bool `json:"enable"` + Bundle string `json:"bundle"` + Unload bool `json:"unload"` } type TopicStats struct { diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index f6c9b432cd..20711a0725 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -17,6 +17,10 @@ package pulsar +import ( + "strconv" +) + type Namespaces interface { // Get the list of all the namespaces for a certain tenant GetNamespaces(tenant string) ([]string, error) @@ -44,6 +48,74 @@ type Namespaces interface { // Delete an existing bundle in a namespace DeleteNamespaceBundle(namespace string, bundleRange string) error + + // Set the messages Time to Live for all the topics within a namespace + SetNamespaceMessageTTL(namespace string, ttlInSeconds int) error + + // Get the message TTL for a namespace + GetNamespaceMessageTTL(namespace string) (int, error) + + // Get the retention configuration for a namespace + GetRetention(namespace string) (*RetentionPolicies, error) + + // Set the retention configuration for all the topics on a namespace + SetRetention(namespace string, policy RetentionPolicies) error + + // Get backlog quota map on a namespace + GetBacklogQuotaMap(namespace string) (map[BacklogQuotaType]BacklogQuota, error) + + // Set a backlog quota for all the topics on a namespace + SetBacklogQuota(namespace string, backlogQuota BacklogQuota) error + + // Remove a backlog quota policy from a namespace + RemoveBacklogQuota(namespace string) error + + // Get the replication clusters for a namespace + GetNamespaceReplicationClusters(namespace string) ([]string, error) + + // Set the replication clusters for a namespace + SetNamespaceReplicationClusters(namespace string, clusterIds []string) error + + // Set anti-affinity group name for a namespace + SetNamespaceAntiAffinityGroup(namespace string, namespaceAntiAffinityGroup string) error + + // Get all namespaces that grouped with given anti-affinity group + GetAntiAffinityNamespaces(tenant, cluster, namespaceAntiAffinityGroup string) ([]string, error) + + // Get anti-affinity group name for a namespace + GetNamespaceAntiAffinityGroup(namespace string) (string, error) + + // Delete anti-affinity group name for a namespace + DeleteNamespaceAntiAffinityGroup(namespace string) error + + // Set the deduplication status for all topics within a namespace + // When deduplication is enabled, the broker will prevent to store the same message multiple times + SetDeduplicationStatus(namespace string, enableDeduplication bool) error + + // Set the persistence configuration for all the topics on a namespace + SetPersistence(namespace string, persistence PersistencePolicies) error + + // Get the persistence configuration for a namespace + GetPersistence(namespace string) (*PersistencePolicies, error) + + // Set bookie affinity group for a namespace to isolate namespace write to bookies that are + // part of given affinity group + SetBookieAffinityGroup(namespace string, bookieAffinityGroup BookieAffinityGroupData) error + + // Delete bookie affinity group configured for a namespace + DeleteBookieAffinityGroup(namespace string) error + + // Get bookie affinity group configured for a namespace + GetBookieAffinityGroup(namespace string) (*BookieAffinityGroupData, error) + + // Unload a namespace from the current serving broker + Unload(namespace string) error + + // Unload namespace bundle + UnloadNamespaceBundle(namespace, bundle string) error + + // Split namespace bundle + SplitNamespaceBundle(namespace, bundle string, unloadSplitBundles bool) error } type namespaces struct { @@ -138,3 +210,222 @@ func (n *namespaces) DeleteNamespaceBundle(namespace string, bundleRange string) endpoint := n.client.endpoint(n.basePath, ns.String(), bundleRange) return n.client.delete(endpoint, nil) } + +func (n *namespaces) GetNamespaceMessageTTL(namespace string) (int, error) { + var ttl int + nsName, err := GetNamespaceName(namespace) + if err != nil { + return 0, err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "messageTTL") + err = n.client.get(endpoint, &ttl) + return ttl, err +} + +func (n *namespaces) SetNamespaceMessageTTL(namespace string, ttlInSeconds int) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + + endpoint := n.client.endpoint(n.basePath, nsName.String(), "messageTTL") + return n.client.post(endpoint, &ttlInSeconds, nil) +} + +func (n *namespaces) SetRetention(namespace string, policy RetentionPolicies) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "retention") + return n.client.post(endpoint, &policy, nil) +} + +func (n *namespaces) GetRetention(namespace string) (*RetentionPolicies, error) { + var policy RetentionPolicies + nsName, err := GetNamespaceName(namespace) + if err != nil { + return nil, err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "retention") + err = n.client.get(endpoint, &policy) + return &policy, err +} + +func (n *namespaces) GetBacklogQuotaMap(namespace string) (map[BacklogQuotaType]BacklogQuota, error) { + var backlogQuotaMap map[BacklogQuotaType]BacklogQuota + nsName, err := GetNamespaceName(namespace) + if err != nil { + return nil, err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuotaMap") + err = n.client.get(endpoint, &backlogQuotaMap) + return backlogQuotaMap, err +} + +func (n *namespaces) SetBacklogQuota(namespace string, backlogQuota BacklogQuota) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuota") + return n.client.post(endpoint, &backlogQuota, nil) +} + +func (n *namespaces) RemoveBacklogQuota(namespace string) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuota") + params := map[string]string{ + "backlogQuotaType": string(DestinationStorage), + } + return n.client.deleteWithQueryParams(endpoint, nil, params) +} + +func (n *namespaces) GetNamespaceReplicationClusters(namespace string) ([]string, error) { + var data []string + nsName, err := GetNamespaceName(namespace) + if err != nil { + return nil, err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "replication") + err = n.client.get(endpoint, &data) + return data, err +} + +func (n *namespaces) SetNamespaceReplicationClusters(namespace string, clusterIds []string) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "replication") + return n.client.post(endpoint, &clusterIds, nil) +} + +func (n *namespaces) SetNamespaceAntiAffinityGroup(namespace string, namespaceAntiAffinityGroup string) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "antiAffinity") + return n.client.post(endpoint, namespaceAntiAffinityGroup, nil) +} + +func (n *namespaces) GetAntiAffinityNamespaces(tenant, cluster, namespaceAntiAffinityGroup string) ([]string, error) { + var data []string + endpoint := n.client.endpoint(n.basePath, cluster, "antiAffinity", namespaceAntiAffinityGroup) + params := map[string]string{ + "property": tenant, + } + _, err := n.client.getWithQueryParams(endpoint, &data, params, false) + return data, err +} + +func (n *namespaces) GetNamespaceAntiAffinityGroup(namespace string) (string, error) { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return "", err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "antiAffinity") + data, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + return string(data), err +} + +func (n *namespaces) DeleteNamespaceAntiAffinityGroup(namespace string) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "antiAffinity") + return n.client.delete(endpoint, nil) +} + +func (n *namespaces) SetDeduplicationStatus(namespace string, enableDeduplication bool) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "deduplication") + return n.client.post(endpoint, enableDeduplication, nil) +} + +func (n *namespaces) SetPersistence(namespace string, persistence PersistencePolicies) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence") + return n.client.post(endpoint, &persistence, nil) +} + +func (n *namespaces) SetBookieAffinityGroup(namespace string, bookieAffinityGroup BookieAffinityGroupData) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence", "bookieAffinity") + return n.client.post(endpoint, &bookieAffinityGroup, nil) +} + +func (n *namespaces) DeleteBookieAffinityGroup(namespace string) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence", "bookieAffinity") + return n.client.delete(endpoint, nil) +} + +func (n *namespaces) GetBookieAffinityGroup(namespace string) (*BookieAffinityGroupData, error) { + var data BookieAffinityGroupData + nsName, err := GetNamespaceName(namespace) + if err != nil { + return nil, err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence", "bookieAffinity") + err = n.client.get(endpoint, &data) + return &data, err +} + +func (n *namespaces) GetPersistence(namespace string) (*PersistencePolicies, error) { + var persistence PersistencePolicies + nsName, err := GetNamespaceName(namespace) + if err != nil { + return nil, err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence") + err = n.client.get(endpoint, &persistence) + return &persistence, err +} + +func (n *namespaces) Unload(namespace string) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), "unload") + return n.client.put(endpoint, "", nil) +} + +func (n *namespaces) UnloadNamespaceBundle(namespace, bundle string) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), bundle, "unload") + return n.client.put(endpoint, "", nil) +} + +func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitBundles bool) error { + nsName, err := GetNamespaceName(namespace) + if err != nil { + return err + } + endpoint := n.client.endpoint(n.basePath, nsName.String(), bundle, "split") + params := map[string]string{ + "unload": strconv.FormatBool(unloadSplitBundles), + } + return n.client.putWithQueryParams(endpoint, "", nil, params) +} diff --git a/pulsaradmin/pkg/pulsar/persistence_policies.go b/pulsaradmin/pkg/pulsar/persistence_policies.go index e6dfb7ddb5..a8739c1e8d 100644 --- a/pulsaradmin/pkg/pulsar/persistence_policies.go +++ b/pulsaradmin/pkg/pulsar/persistence_policies.go @@ -18,13 +18,13 @@ package pulsar type PersistencePolicies struct { - BookkeeperEnsemble int - BookkeeperWriteQuorum int - BookkeeperAckQuorum int - ManagedLedgerMaxMarkDeleteRate float64 + BookkeeperEnsemble int `json:"bookkeeperEnsemble"` + BookkeeperWriteQuorum int `json:"bookkeeperWriteQuorum"` + BookkeeperAckQuorum int `json:"bookkeeperAckQuorum"` + ManagedLedgerMaxMarkDeleteRate float64 `json:"managedLedgerMaxMarkDeleteRate"` } -func NewPersistencePolicies() *PersistencePolicies { +func NewDefaultPersistencePolicies() *PersistencePolicies { return &PersistencePolicies{ BookkeeperEnsemble: 2, BookkeeperWriteQuorum: 2, @@ -32,3 +32,24 @@ func NewPersistencePolicies() *PersistencePolicies { ManagedLedgerMaxMarkDeleteRate: 0.0, } } + +func NewPersistencePolicies(bookkeeperEnsemble, bookkeeperWriteQuorum, bookkeeperAckQuorum int, managedLedgerMaxMarkDeleteRate float64) PersistencePolicies { + return PersistencePolicies{ + BookkeeperEnsemble: bookkeeperEnsemble, + BookkeeperWriteQuorum: bookkeeperWriteQuorum, + BookkeeperAckQuorum: bookkeeperAckQuorum, + ManagedLedgerMaxMarkDeleteRate: managedLedgerMaxMarkDeleteRate, + } +} + +type BookieAffinityGroupData struct { + BookkeeperAffinityGroupPrimary string `json:"bookkeeperAffinityGroupPrimary"` + BookkeeperAffinityGroupSecondary string `json:"bookkeeperAffinityGroupSecondary"` +} + +func NewBookieAffinityGroupData(bookkeeperAffinityGroupPrimary, bookkeeperAffinityGroupSecondary string) BookieAffinityGroupData { + return BookieAffinityGroupData{ + BookkeeperAffinityGroupPrimary: bookkeeperAffinityGroupPrimary, + BookkeeperAffinityGroupSecondary: bookkeeperAffinityGroupSecondary, + } +} diff --git a/pulsaradmin/pkg/pulsar/retention_policies.go b/pulsaradmin/pkg/pulsar/retention_policies.go index c25214c083..23e3d7c736 100644 --- a/pulsaradmin/pkg/pulsar/retention_policies.go +++ b/pulsaradmin/pkg/pulsar/retention_policies.go @@ -18,6 +18,13 @@ package pulsar type RetentionPolicies struct { - RetentionTimeInMinutes int - RetentionSizeInMB int64 + RetentionTimeInMinutes int `json:"retentionTimeInMinutes"` + RetentionSizeInMB int64 `json:"retentionSizeInMB"` +} + +func NewRetentionPolicies(retentionTimeInMinutes int, retentionSizeInMB int) RetentionPolicies { + return RetentionPolicies{ + RetentionTimeInMinutes: retentionTimeInMinutes, + RetentionSizeInMB: int64(retentionSizeInMB), + } } From 2efdd3c26d4690daf5a0fe44f0ee066934b907a9 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 19 Sep 2019 23:33:55 +0800 Subject: [PATCH 119/348] Add topic permissions commands (streamnative/pulsarctl#48) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## OUTPUT *get-permissions`* ``` ➜ pulsarctl git:(topic-permission) ./pulsarctl topic get-permissions -h USED FOR: This command is used for getting the permissions on a topic. REQUIRED PERMISSION: This command requires namespace admin permissions. EXAMPLES: #Get the permissions on a topic pulsarctl topic get-permissions OUTPUT: #normal output { "": [ "" ] } #the topic name is not specified [✖] only one argument is allowed to be used as a name #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics get-permissions [flags] Aliases: get-permissions, get ``` *grant-permissions* ``` ➜ pulsarctl git:(topic-permission) ./pulsarctl topic grant-permissions -h USED FOR: This command is used for granting permissions to a client role on a single topic. REQUIRED PERMISSION: This command requires namespace admin permissions. EXAMPLES: #Grant permissions to a client on a single topic pulsarctl topic grant-permissions --role --actions --actions OUTPUT: #normal output Grant role %s and actions %v to the topic %s successfully #the topic name is not specified [✖] only one argument is allowed to be used as a name #the specified role is empty Invalid role name #the specified actions is not allowed. The auth action only can be specified as 'produce', 'consume', or 'functions'. Invalid auth action '' #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics grant-permissions [flags] Aliases: grant-permissions, grant ``` *revoke-permissions* ``` ➜ pulsarctl git:(topic-permission) ./pulsarctl topic revoke-permissions -h USED FOR: This command is used for revoking permissions on a topic. REQUIRED PERMISSION: This command requires namespace admin permissions. EXAMPLES: OUTPUT: #normal output Revoke permissions for the role to the topic successfully #the specified role is empty Invalid role name #the topic name is not specified [✖] only one argument is allowed to be used as a name #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics revoke-permissions [flags] Aliases: revoke-permissions, revoke ``` --- pulsaradmin/pkg/pulsar/admin.go | 2 +- pulsaradmin/pkg/pulsar/auth_action.go | 29 ++++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/auth_polices.go | 8 ------- pulsaradmin/pkg/pulsar/topic.go | 24 +++++++++++++++++++++ 4 files changed, 54 insertions(+), 9 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/auth_action.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index f3508d98fc..d52754d329 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -55,9 +55,9 @@ type Client interface { Clusters() Clusters Functions() Functions Tenants() Tenants + Topics() Topics Sources() Sources Sinks() Sinks - Topics() Topics Namespaces() Namespaces Schemas() Schema } diff --git a/pulsaradmin/pkg/pulsar/auth_action.go b/pulsaradmin/pkg/pulsar/auth_action.go new file mode 100644 index 0000000000..ed42d52b6b --- /dev/null +++ b/pulsaradmin/pkg/pulsar/auth_action.go @@ -0,0 +1,29 @@ +package pulsar + +import "github.com/pkg/errors" + +type AuthAction string + +const ( + produce AuthAction = "produce" + consume AuthAction = "consume" + functionsAuth AuthAction = "functions" +) + +func ParseAuthAction(action string) (AuthAction, error) { + switch action { + case "produce": + return produce, nil + case "consume": + return consume, nil + case "functions": + return functionsAuth, nil + default: + return "", errors.Errorf("The auth action only can be specified as 'produce', "+ + "'consume', or 'functions'. Invalid auth action '%s'", action) + } +} + +func (a AuthAction) String() string { + return string(a) +} diff --git a/pulsaradmin/pkg/pulsar/auth_polices.go b/pulsaradmin/pkg/pulsar/auth_polices.go index 0200f11466..96a906cdbb 100644 --- a/pulsaradmin/pkg/pulsar/auth_polices.go +++ b/pulsaradmin/pkg/pulsar/auth_polices.go @@ -30,11 +30,3 @@ func NewAuthPolicies() *AuthPolicies { SubscriptionAuthRoles: make(map[string][]string), } } - -type AuthAction string - -const ( - produce AuthAction = "produce" - consume AuthAction = "consume" - function AuthAction = "functions" -) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index d193e6ed2a..fabe198538 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -11,6 +11,9 @@ type Topics interface { Update(TopicName, int) error GetMetadata(TopicName) (PartitionedTopicMetadata, error) List(NameSpaceName) ([]string, []string, error) + GetPermissions(TopicName) (map[string][]AuthAction, error) + GrantPermission(TopicName, string, []AuthAction) error + RevokePermission(TopicName, string) error Lookup(TopicName) (LookupData, error) GetBundleRange(TopicName) (string, error) GetLastMessageId(TopicName) (MessageId, error) @@ -112,6 +115,27 @@ func (t *topics) getTopics(endpoint string, out chan<- []string, err chan<- erro out <- topics } +func (t *topics) GetPermissions(topic TopicName) (map[string][]AuthAction, error) { + var permissions map[string][]AuthAction + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions") + err := t.client.get(endpoint, &permissions) + return permissions, err +} + +func (t *topics) GrantPermission(topic TopicName, role string, action []AuthAction) error { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions", role) + var s []string + for _, v := range action { + s = append(s, v.String()) + } + return t.client.post(endpoint, s, nil) +} + +func (t *topics) RevokePermission(topic TopicName, role string) error { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions", role) + return t.client.delete(endpoint, nil) +} + func (t *topics) Lookup(topic TopicName) (LookupData, error) { var lookup LookupData endpoint := fmt.Sprintf("%s/%s", t.lookupPath, topic.GetRestPath()) From 13c4241eae84693f3c9a2999f47ae2e95f8cdccd Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 24 Sep 2019 14:10:46 +0800 Subject: [PATCH 120/348] Add license header check for all files (streamnative/pulsarctl#69) Fixes: streamnative/pulsarctl#65 --- pulsaradmin/pkg/pulsar/Tenant.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/admin.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/api_version.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/api_version_test.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/auth_action.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/cluster.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/descriptions.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/descriptions_test.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/errors.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/message_id.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/namespace_name.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/namespace_name_test.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/topic.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/topic_domain.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/topic_name.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/topic_name_test.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/utils.go | 17 +++++++++++++++++ 17 files changed, 289 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/Tenant.go b/pulsaradmin/pkg/pulsar/Tenant.go index b41627b649..1b3a67e069 100644 --- a/pulsaradmin/pkg/pulsar/Tenant.go +++ b/pulsaradmin/pkg/pulsar/Tenant.go @@ -1,3 +1,20 @@ +// 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 pulsar type Tenants interface { diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index d52754d329..167202cc8b 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -1,3 +1,20 @@ +// 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 pulsar import ( diff --git a/pulsaradmin/pkg/pulsar/api_version.go b/pulsaradmin/pkg/pulsar/api_version.go index fd239798c7..07c52972a4 100644 --- a/pulsaradmin/pkg/pulsar/api_version.go +++ b/pulsaradmin/pkg/pulsar/api_version.go @@ -1,3 +1,20 @@ +// 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 pulsar type ApiVersion int diff --git a/pulsaradmin/pkg/pulsar/api_version_test.go b/pulsaradmin/pkg/pulsar/api_version_test.go index 8de8fac7ef..2a0c3a88f7 100644 --- a/pulsaradmin/pkg/pulsar/api_version_test.go +++ b/pulsaradmin/pkg/pulsar/api_version_test.go @@ -1,3 +1,20 @@ +// 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 pulsar import ( diff --git a/pulsaradmin/pkg/pulsar/auth_action.go b/pulsaradmin/pkg/pulsar/auth_action.go index ed42d52b6b..1bf9fe1e68 100644 --- a/pulsaradmin/pkg/pulsar/auth_action.go +++ b/pulsaradmin/pkg/pulsar/auth_action.go @@ -1,3 +1,20 @@ +// 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 pulsar import "github.com/pkg/errors" diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index 55661abd21..fcd3cd700b 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -1,3 +1,20 @@ +// 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 pulsar // Clusters is used to access the cluster endpoints. diff --git a/pulsaradmin/pkg/pulsar/descriptions.go b/pulsaradmin/pkg/pulsar/descriptions.go index 8cb29804a8..991108ed9d 100644 --- a/pulsaradmin/pkg/pulsar/descriptions.go +++ b/pulsaradmin/pkg/pulsar/descriptions.go @@ -1,3 +1,20 @@ +// 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 pulsar import "strings" diff --git a/pulsaradmin/pkg/pulsar/descriptions_test.go b/pulsaradmin/pkg/pulsar/descriptions_test.go index fa643185bc..7a0d65f00e 100644 --- a/pulsaradmin/pkg/pulsar/descriptions_test.go +++ b/pulsaradmin/pkg/pulsar/descriptions_test.go @@ -1,3 +1,20 @@ +// 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 pulsar import ( diff --git a/pulsaradmin/pkg/pulsar/errors.go b/pulsaradmin/pkg/pulsar/errors.go index 19cfd2983c..ae3a317cbb 100644 --- a/pulsaradmin/pkg/pulsar/errors.go +++ b/pulsaradmin/pkg/pulsar/errors.go @@ -1,3 +1,20 @@ +// 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 pulsar import "fmt" diff --git a/pulsaradmin/pkg/pulsar/message_id.go b/pulsaradmin/pkg/pulsar/message_id.go index c24aabd4c7..a86a830c5d 100644 --- a/pulsaradmin/pkg/pulsar/message_id.go +++ b/pulsaradmin/pkg/pulsar/message_id.go @@ -1,3 +1,20 @@ +// 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 pulsar type MessageId struct { diff --git a/pulsaradmin/pkg/pulsar/namespace_name.go b/pulsaradmin/pkg/pulsar/namespace_name.go index b55a34d6d2..3e1c2e5b49 100644 --- a/pulsaradmin/pkg/pulsar/namespace_name.go +++ b/pulsaradmin/pkg/pulsar/namespace_name.go @@ -1,3 +1,20 @@ +// 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 pulsar import ( diff --git a/pulsaradmin/pkg/pulsar/namespace_name_test.go b/pulsaradmin/pkg/pulsar/namespace_name_test.go index b1c7c2c8ef..fc7b04f839 100644 --- a/pulsaradmin/pkg/pulsar/namespace_name_test.go +++ b/pulsaradmin/pkg/pulsar/namespace_name_test.go @@ -1,3 +1,20 @@ +// 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 pulsar import ( diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index fabe198538..649729f689 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -1,3 +1,20 @@ +// 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 pulsar import ( diff --git a/pulsaradmin/pkg/pulsar/topic_domain.go b/pulsaradmin/pkg/pulsar/topic_domain.go index 501ebcbe65..a54d1a1abf 100644 --- a/pulsaradmin/pkg/pulsar/topic_domain.go +++ b/pulsaradmin/pkg/pulsar/topic_domain.go @@ -1,3 +1,20 @@ +// 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 pulsar import "github.com/pkg/errors" diff --git a/pulsaradmin/pkg/pulsar/topic_name.go b/pulsaradmin/pkg/pulsar/topic_name.go index 5561c2b431..5c27eb1501 100644 --- a/pulsaradmin/pkg/pulsar/topic_name.go +++ b/pulsaradmin/pkg/pulsar/topic_name.go @@ -1,3 +1,20 @@ +// 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 pulsar import ( diff --git a/pulsaradmin/pkg/pulsar/topic_name_test.go b/pulsaradmin/pkg/pulsar/topic_name_test.go index 7cb45a49cb..a7be0d03f3 100644 --- a/pulsaradmin/pkg/pulsar/topic_name_test.go +++ b/pulsaradmin/pkg/pulsar/topic_name_test.go @@ -1,3 +1,20 @@ +// 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 pulsar import ( diff --git a/pulsaradmin/pkg/pulsar/utils.go b/pulsaradmin/pkg/pulsar/utils.go index 740ed4b3ab..e408745167 100644 --- a/pulsaradmin/pkg/pulsar/utils.go +++ b/pulsaradmin/pkg/pulsar/utils.go @@ -1,3 +1,20 @@ +// 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 pulsar import ( From 896c915128175db65691efe3e5a1bb75af477db9 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 24 Sep 2019 14:13:39 +0800 Subject: [PATCH 121/348] Add command topic `internal-info` (streamnative/pulsarctl#51) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### OUTPUT *internal-info* ``` ➜ pulsarctl git:(topic-info) ✗ ./pulsarctl topic internal-info -h USED FOR: This command is used for getting the internal info of a topic which has messages or subscriptions. REQUIRED PERMISSION: This command requires tenant admin permissions. EXAMPLES: #Get the internal info of a topic pulsarctl topic internal-info OUTPUT: #normal output { "version": 1, "creationDate": "", "modificationData": "", "ledgers": [ { "ledgerId": 0, "entries": 0, "size": 0, "timestamp": 0 } ], "terminatedPosition": { "ledgerId": 0, "entryId": 0 }, "cursors": { "hello": { "version": 0, "creationDate": "", "modificationDate": "", "cursorsLedgerId": 0, "markDelete": { "ledgerId": 0, "entryId": 0 }, "individualDeletedMessages": null, "Properties": null } } } #the topic name is not specified [✖] only one argument is allowed to be used as a name #the specified topic does not existing [✖] code: 500 reason: Unknown pulsar error #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics internal-info [flags] ``` --- pulsaradmin/pkg/pulsar/admin.go | 1 + pulsaradmin/pkg/pulsar/data.go | 31 ++++++++++++++++++++++++++++++- pulsaradmin/pkg/pulsar/topic.go | 8 ++++++++ 3 files changed, 39 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 167202cc8b..bd49080ef4 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -462,6 +462,7 @@ func responseError(resp *http.Response) error { json.Unmarshal(body, &e) e.Code = resp.StatusCode + if e.Reason == "" { e.Reason = unknownErrorReason } diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index d9370a33b7..18cbf29949 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -151,7 +151,15 @@ type PartitionedTopicMetadata struct { Partitions int `json:"partitions"` } -type NamespacesData struct { +type ManagedLedgerInfo struct { + Version int `json:"version"` + CreationDate string `json:"creationDate"` + ModificationData string `json:"modificationData"` + Ledgers []LedgerInfo `json:"ledgers"` + TerminatedPosition PositionInfo `json:"terminatedPosition"` + Cursors map[string]CursorInfo `json:"cursors"` +} + type NamespacesData struct { NumBundles int `json:"numBundles"` Clusters []string `json:"clusters"` ClusterIds string `json:"clusterIds"` @@ -253,6 +261,27 @@ type LedgerInfo struct { LedgerId int64 `json:"ledgerId"` Entries int64 `json:"entries"` Size int64 `json:"size"` + Timestamp int64 `json:"timestamp"` +} + +type CursorInfo struct { + Version int `json:"version"` + CreationDate string `json:"creationDate"` + ModificationDate string `json:"modificationDate"` + CursorsLedgerId int64 `json:"cursorsLedgerId"` + MarkDelete PositionInfo `json:"markDelete"` + IndividualDeletedMessages []MessageRangeInfo `json:"individualDeletedMessages"` + Properties map[string]int64 +} + +type PositionInfo struct { + LedgerId int64 `json:"ledgerId"` + EntryId int64 `json:"entryId"` +} + +type MessageRangeInfo struct { + From PositionInfo `json:"from"` + To PositionInfo `json:"to"` Offloaded bool `json:"offloaded"` } diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 649729f689..0b1260a508 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -28,6 +28,7 @@ type Topics interface { Update(TopicName, int) error GetMetadata(TopicName) (PartitionedTopicMetadata, error) List(NameSpaceName) ([]string, []string, error) + GetInternalInfo(TopicName) (ManagedLedgerInfo, error) GetPermissions(TopicName) (map[string][]AuthAction, error) GrantPermission(TopicName, string, []AuthAction) error RevokePermission(TopicName, string) error @@ -132,6 +133,13 @@ func (t *topics) getTopics(endpoint string, out chan<- []string, err chan<- erro out <- topics } +func (t *topics) GetInternalInfo(topic TopicName) (ManagedLedgerInfo, error) { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "internal-info") + var info ManagedLedgerInfo + err := t.client.get(endpoint, &info) + return info, err +} + func (t *topics) GetPermissions(topic TopicName) (map[string][]AuthAction, error) { var permissions map[string][]AuthAction endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions") From 1939c6ca4337aada94be8f34fa778f0497b0bd4f Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 24 Sep 2019 15:23:10 +0800 Subject: [PATCH 122/348] Add command unsubscribe and clearBacklog and command set and get for dispatch and subscribe and replicator rate (streamnative/pulsarctl#68) * Add command unsubscribe and clearBacklog and command set and get for dispatch and subscirbe and replicator rate -- *Modifications* - Add command clear-backlog - Add command set-dispatch-rate - Add command get-dispatch-rate - Add command messages-encryption - Add command set-replicator-dispatch-rate - Add command get-replicator-dispatch-rate - Add command set-subscribe-rate - Add command get-subscribe-rate - Add command set-subscription-auth-mode - Add command set-subscription-dispatch-rate - Add command get-subscription-dispatch-rate - Add command unsubscribe --- pulsaradmin/pkg/pulsar/dispatch_rate.go | 10 +- pulsaradmin/pkg/pulsar/namespace.go | 137 ++++++++++++++++++ pulsaradmin/pkg/pulsar/policies.go | 6 - .../pkg/pulsar/subscription_auth_mode.go | 42 ++++++ 4 files changed, 184 insertions(+), 11 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/subscription_auth_mode.go diff --git a/pulsaradmin/pkg/pulsar/dispatch_rate.go b/pulsaradmin/pkg/pulsar/dispatch_rate.go index 3813b03b27..7ccd3d758a 100644 --- a/pulsaradmin/pkg/pulsar/dispatch_rate.go +++ b/pulsaradmin/pkg/pulsar/dispatch_rate.go @@ -18,9 +18,9 @@ package pulsar type DispatchRate struct { - DispatchThrottlingRateInMsg int - DispatchThrottlingRateInByte int64 - RatePeriodInSecond int + DispatchThrottlingRateInMsg int `json:"dispatchThrottlingRateInMsg"` + DispatchThrottlingRateInByte int64 `json:"dispatchThrottlingRateInByte"` + RatePeriodInSecond int `json:"ratePeriodInSecond"` } func NewDispatchRate() *DispatchRate { @@ -32,8 +32,8 @@ func NewDispatchRate() *DispatchRate { } type SubscribeRate struct { - SubscribeThrottlingRatePerConsumer int - RatePeriodInSecond int + SubscribeThrottlingRatePerConsumer int `json:"subscribeThrottlingRatePerConsumer"` + RatePeriodInSecond int `json:"ratePeriodInSecond"` } func NewSubscribeRate() *SubscribeRate { diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index 20711a0725..f3c42a7d49 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -18,6 +18,7 @@ package pulsar import ( + "net/url" "strconv" ) @@ -116,6 +117,54 @@ type Namespaces interface { // Split namespace bundle SplitNamespaceBundle(namespace, bundle string, unloadSplitBundles bool) error + + // Set the given subscription auth mode on all topics on a namespace + SetSubscriptionAuthMode(namespace NameSpaceName, mode SubscriptionAuthMode) error + + // Set the encryption required status for all topics within a namespace + SetEncryptionRequiredStatus(namespace NameSpaceName, encrypt bool) error + + // Unsubscribe the given subscription on all topics on a namespace + UnsubscribeNamespace(namespace NameSpaceName, sName string) error + + // Unsubscribe the given subscription on all topics on a namespace bundle + UnsubscribeNamespaceBundle(namespace NameSpaceName, bundle, sName string) error + + // Clear backlog for a given subscription on all topics on a namespace bundle + ClearNamespaceBundleBacklogForSubscription(namespace NameSpaceName, bundle, sName string) error + + // Clear backlog for all topics on a namespace bundle + ClearNamespaceBundleBacklog(namespace NameSpaceName, bundle string) error + + // Clear backlog for a given subscription on all topics on a namespace + ClearNamespaceBacklogForSubscription(namespace NameSpaceName, sName string) error + + // Clear backlog for all topics on a namespace + ClearNamespaceBacklog(namespace NameSpaceName) error + + // Set replicator-message-dispatch-rate (Replicators under this namespace can dispatch this many messages per second) + SetReplicatorDispatchRate(namespace NameSpaceName, rate DispatchRate) error + + // Get replicator-message-dispatch-rate (Replicators under this namespace can dispatch this many messages per second) + GetReplicatorDispatchRate(namespace NameSpaceName) (DispatchRate, error) + + // Set subscription-message-dispatch-rate (subscriptions under this namespace can dispatch this many messages per second) + SetSubscriptionDispatchRate(namespace NameSpaceName, rate DispatchRate) error + + // Get subscription-message-dispatch-rate (subscriptions under this namespace can dispatch this many messages per second) + GetSubscriptionDispatchRate(namespace NameSpaceName) (DispatchRate, error) + + // Set namespace-subscribe-rate (topics under this namespace will limit by subscribeRate) + SetSubscribeRate(namespace NameSpaceName, rate SubscribeRate) error + + // Get namespace-subscribe-rate (topics under this namespace allow subscribe times per consumer in a period) + GetSubscribeRate(namespace NameSpaceName) (SubscribeRate, error) + + // Set message-dispatch-rate (topics under this namespace can dispatch this many messages per second) + SetDispatchRate(namespace NameSpaceName, rate DispatchRate) error + + // Get message-dispatch-rate (topics under this namespace can dispatch this many messages per second) + GetDispatchRate(namespace NameSpaceName) (DispatchRate, error) } type namespaces struct { @@ -429,3 +478,91 @@ func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitB } return n.client.putWithQueryParams(endpoint, "", nil, params) } + +func (n *namespaces) SetSubscriptionAuthMode(namespace NameSpaceName, mode SubscriptionAuthMode) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionAuthMode") + return n.client.post(endpoint, mode.String(), nil) +} + +func (n *namespaces) SetEncryptionRequiredStatus(namespace NameSpaceName, encrypt bool) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "encryptionRequired") + return n.client.post(endpoint, strconv.FormatBool(encrypt), nil) +} + +func (n *namespaces) UnsubscribeNamespace(namespace NameSpaceName, sName string) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "unsubscribe", url.QueryEscape(sName)) + return n.client.post(endpoint, "", nil) +} + +func (n *namespaces) UnsubscribeNamespaceBundle(namespace NameSpaceName, bundle, sName string) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "unsubscribe", url.QueryEscape(sName)) + return n.client.post(endpoint, "", nil) +} + +func (n *namespaces) ClearNamespaceBundleBacklogForSubscription(namespace NameSpaceName, bundle, sName string) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog", url.QueryEscape(sName)) + return n.client.post(endpoint, "", nil) +} + +func (n *namespaces) ClearNamespaceBundleBacklog(namespace NameSpaceName, bundle string) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog") + return n.client.post(endpoint, "", nil) +} + +func (n *namespaces) ClearNamespaceBacklogForSubscription(namespace NameSpaceName, sName string) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "clearBacklog", url.QueryEscape(sName)) + return n.client.post(endpoint, "", nil) +} + +func (n *namespaces) ClearNamespaceBacklog(namespace NameSpaceName) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "clearBacklog") + return n.client.post(endpoint, "", nil) +} + +func (n *namespaces) SetReplicatorDispatchRate(namespace NameSpaceName, rate DispatchRate) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "replicatorDispatchRate") + return n.client.post(endpoint, rate, nil) +} + +func (n *namespaces) GetReplicatorDispatchRate(namespace NameSpaceName) (DispatchRate, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "replicatorDispatchRate") + var rate DispatchRate + err := n.client.get(endpoint, &rate) + return rate, err +} + +func (n *namespaces) SetSubscriptionDispatchRate(namespace NameSpaceName, rate DispatchRate) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionDispatchRate") + return n.client.post(endpoint, rate, nil) +} + +func (n *namespaces) GetSubscriptionDispatchRate(namespace NameSpaceName) (DispatchRate, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionDispatchRate") + var rate DispatchRate + err := n.client.get(endpoint, &rate) + return rate, err +} + +func (n *namespaces) SetSubscribeRate(namespace NameSpaceName, rate SubscribeRate) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscribeRate") + return n.client.post(endpoint, rate, nil) +} + +func (n *namespaces) GetSubscribeRate(namespace NameSpaceName) (SubscribeRate, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscribeRate") + var rate SubscribeRate + err := n.client.get(endpoint, &rate) + return rate, err +} + +func (n *namespaces) SetDispatchRate(namespace NameSpaceName, rate DispatchRate) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "dispatchRate") + return n.client.post(endpoint, rate, nil) +} + +func (n *namespaces) GetDispatchRate(namespace NameSpaceName) (DispatchRate, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "dispatchRate") + var rate DispatchRate + err := n.client.get(endpoint, &rate) + return rate, err +} diff --git a/pulsaradmin/pkg/pulsar/policies.go b/pulsaradmin/pkg/pulsar/policies.go index d8336db522..f664335144 100644 --- a/pulsaradmin/pkg/pulsar/policies.go +++ b/pulsaradmin/pkg/pulsar/policies.go @@ -74,9 +74,3 @@ func NewDefaultPolicies() *Policies { } } -type SubscriptionAuthMode string - -const ( - None SubscriptionAuthMode = "None" - Prefix SubscriptionAuthMode = "Prefix" -) diff --git a/pulsaradmin/pkg/pulsar/subscription_auth_mode.go b/pulsaradmin/pkg/pulsar/subscription_auth_mode.go new file mode 100644 index 0000000000..f55ab1997a --- /dev/null +++ b/pulsaradmin/pkg/pulsar/subscription_auth_mode.go @@ -0,0 +1,42 @@ +// 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 pulsar + +import "github.com/pkg/errors" + +type SubscriptionAuthMode string + +const ( + None SubscriptionAuthMode = "None" + Prefix SubscriptionAuthMode = "Prefix" +) + +func ParseSubscriptionAuthMode(s string) (SubscriptionAuthMode, error) { + switch s { + case "None": + return None, nil + case "Prefix": + return Prefix, nil + default: + return "", errors.New("Invalid subscription auth mode") + } +} + +func (s SubscriptionAuthMode) String() string { + return string(s) +} From 483bf1690000fafbd5749ae3b8aaa3cf4562ec05 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 3 Oct 2019 12:21:37 +0800 Subject: [PATCH 123/348] Add golangci-lint to check project style (streamnative/pulsarctl#78) *Motivation* Make all project file consistent *Modifications* Format all project files --- pulsaradmin/pkg/auth/tls.go | 14 +- pulsaradmin/pkg/pulsar/Tenant.go | 10 +- pulsaradmin/pkg/pulsar/WindowConfing.go | 25 +- pulsaradmin/pkg/pulsar/admin.go | 99 ++- pulsaradmin/pkg/pulsar/api_version.go | 10 +- pulsaradmin/pkg/pulsar/api_version_test.go | 5 +- pulsaradmin/pkg/pulsar/backlog_quota.go | 4 +- pulsaradmin/pkg/pulsar/cluster.go | 14 +- pulsaradmin/pkg/pulsar/connectorDefinition.go | 20 +- pulsaradmin/pkg/pulsar/consumerConfig.go | 8 +- pulsaradmin/pkg/pulsar/data.go | 196 +++-- pulsaradmin/pkg/pulsar/descriptions.go | 4 +- pulsaradmin/pkg/pulsar/descriptions_test.go | 3 +- pulsaradmin/pkg/pulsar/errors.go | 4 +- pulsaradmin/pkg/pulsar/functionConfg.go | 90 +-- pulsaradmin/pkg/pulsar/function_status.go | 8 +- pulsaradmin/pkg/pulsar/functions.go | 66 +- pulsaradmin/pkg/pulsar/functions_stats.go | 6 +- pulsaradmin/pkg/pulsar/message_id.go | 8 +- pulsaradmin/pkg/pulsar/namespace.go | 70 +- pulsaradmin/pkg/pulsar/namespace_name.go | 7 +- pulsaradmin/pkg/pulsar/namespace_name_test.go | 5 +- .../pkg/pulsar/persistence_policies.go | 19 +- pulsaradmin/pkg/pulsar/policies.go | 89 ++- pulsaradmin/pkg/pulsar/resources.go | 45 +- pulsaradmin/pkg/pulsar/schema.go | 4 +- pulsaradmin/pkg/pulsar/schema_strategy.go | 18 +- pulsaradmin/pkg/pulsar/sinkConfig.go | 55 +- pulsaradmin/pkg/pulsar/sink_status.go | 63 +- pulsaradmin/pkg/pulsar/sinks.go | 679 +++++++++--------- pulsaradmin/pkg/pulsar/sourceConfig.go | 40 +- pulsaradmin/pkg/pulsar/source_status.go | 48 +- pulsaradmin/pkg/pulsar/sources.go | 658 ++++++++--------- pulsaradmin/pkg/pulsar/topic.go | 22 +- pulsaradmin/pkg/pulsar/topic_domain.go | 8 +- pulsaradmin/pkg/pulsar/topic_name.go | 30 +- pulsaradmin/pkg/pulsar/topic_name_test.go | 14 +- pulsaradmin/pkg/pulsar/utils.go | 2 +- 38 files changed, 1205 insertions(+), 1265 deletions(-) diff --git a/pulsaradmin/pkg/auth/tls.go b/pulsaradmin/pkg/auth/tls.go index faeead661b..ebe33d6839 100644 --- a/pulsaradmin/pkg/auth/tls.go +++ b/pulsaradmin/pkg/auth/tls.go @@ -19,13 +19,13 @@ package auth import "crypto/tls" -type TlsAuthProvider struct { +type TLSAuthProvider struct { certificatePath string privateKeyPath string } // NewAuthenticationTLSWithParams initialize the authentication provider with map param. -func NewAuthenticationTLSWithParams(params map[string]string) *TlsAuthProvider { +func NewAuthenticationTLSWithParams(params map[string]string) *TLSAuthProvider { return NewAuthenticationTLS( params["tlsCertFile"], params["tlsKeyFile"], @@ -33,24 +33,24 @@ func NewAuthenticationTLSWithParams(params map[string]string) *TlsAuthProvider { } // NewAuthenticationTLS initialize the authentication provider -func NewAuthenticationTLS(certificatePath string, privateKeyPath string) *TlsAuthProvider { - return &TlsAuthProvider{ +func NewAuthenticationTLS(certificatePath string, privateKeyPath string) *TLSAuthProvider { + return &TLSAuthProvider{ certificatePath: certificatePath, privateKeyPath: privateKeyPath, } } -func (p *TlsAuthProvider) Init() error { +func (p *TLSAuthProvider) Init() error { // Try to read certificates immediately to provide better error at startup _, err := p.GetTLSCertificate() return err } -func (p *TlsAuthProvider) Name() string { +func (p *TLSAuthProvider) Name() string { return "tls" } -func (p *TlsAuthProvider) GetTLSCertificate() (*tls.Certificate, error) { +func (p *TLSAuthProvider) GetTLSCertificate() (*tls.Certificate, error) { cert, err := tls.LoadX509KeyPair(p.certificatePath, p.privateKeyPath) return &cert, err } diff --git a/pulsaradmin/pkg/pulsar/Tenant.go b/pulsaradmin/pkg/pulsar/Tenant.go index 1b3a67e069..53d9ba5578 100644 --- a/pulsaradmin/pkg/pulsar/Tenant.go +++ b/pulsaradmin/pkg/pulsar/Tenant.go @@ -26,30 +26,30 @@ type Tenants interface { } type tenants struct { - client *client + client *client basePath string } func (c *client) Tenants() Tenants { return &tenants{ - client: c, + client: c, basePath: "/tenants", } } func (c *tenants) Create(data TenantData) error { endpoint := c.client.endpoint(c.basePath, data.Name) - return c.client.put(endpoint, &data, nil) + return c.client.put(endpoint, &data) } func (c *tenants) Delete(name string) error { endpoint := c.client.endpoint(c.basePath, name) - return c.client.delete(endpoint, nil) + return c.client.delete(endpoint) } func (c *tenants) Update(data TenantData) error { endpoint := c.client.endpoint(c.basePath, data.Name) - return c.client.post(endpoint, &data, nil) + return c.client.post(endpoint, &data) } func (c *tenants) List() ([]string, error) { diff --git a/pulsaradmin/pkg/pulsar/WindowConfing.go b/pulsaradmin/pkg/pulsar/WindowConfing.go index cf96ff0c97..d1235f8d1f 100644 --- a/pulsaradmin/pkg/pulsar/WindowConfing.go +++ b/pulsaradmin/pkg/pulsar/WindowConfing.go @@ -17,23 +17,22 @@ package pulsar -const WindowConfigKey = "__WINDOWCONFIGS__" +const WindowConfigKey = "__WINDOWCONFIGS__" type WindowConfig struct { - WindowLengthCount int - WindowLengthDurationMs int64 - SlidingIntervalCount int - SlidingIntervalDurationMs int64 - LateDataTopic string - MaxLagMs int64 - WatermarkEmitIntervalMs int64 - TimestampExtractorClassName string - ActualWindowFunctionClassName string + WindowLengthCount int + WindowLengthDurationMs int64 + SlidingIntervalCount int + SlidingIntervalDurationMs int64 + LateDataTopic string + MaxLagMs int64 + WatermarkEmitIntervalMs int64 + TimestampExtractorClassName string + ActualWindowFunctionClassName string } func NewDefaultWindowConfing() *WindowConfig { - windowConfig := &WindowConfig{} + windowConfig := &WindowConfig{} - return windowConfig + return windowConfig } - diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index bd49080ef4..4f560e411d 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -40,13 +40,13 @@ const ( // Config is used to configure the admin client type Config struct { - WebServiceUrl string - HttpClient *http.Client - ApiVersion ApiVersion + WebServiceURL string + HTTPClient *http.Client + APIVersion APIVersion - Auth *auth.TlsAuthProvider + Auth *auth.TLSAuthProvider AuthParams string - TlsOptions *TLSOptions + TLSOptions *TLSOptions } type TLSOptions struct { @@ -57,10 +57,10 @@ type TLSOptions struct { // DefaultConfig returns a default configuration for the pulsar admin client func DefaultConfig() *Config { config := &Config{ - WebServiceUrl: DefaultWebServiceURL, - HttpClient: http.DefaultClient, + WebServiceURL: DefaultWebServiceURL, + HTTPClient: http.DefaultClient, - TlsOptions: &TLSOptions{ + TLSOptions: &TLSOptions{ AllowInsecureConnection: false, }, } @@ -80,12 +80,12 @@ type Client interface { } type client struct { - webServiceUrl string + webServiceURL string apiVersion string httpClient *http.Client // TLS config - auth *auth.TlsAuthProvider + auth *auth.TLSAuthProvider authParams string tlsOptions *TLSOptions transport *http.Transport @@ -93,18 +93,18 @@ type client struct { // New returns a new client func New(config *Config) (Client, error) { - if len(config.WebServiceUrl) == 0 { - config.WebServiceUrl = DefaultWebServiceURL + if len(config.WebServiceURL) == 0 { + config.WebServiceURL = DefaultWebServiceURL } c := &client{ - apiVersion: config.ApiVersion.String(), - webServiceUrl: config.WebServiceUrl, + apiVersion: config.APIVersion.String(), + webServiceURL: config.WebServiceURL, } - if strings.HasPrefix(c.webServiceUrl, "https://") { + if strings.HasPrefix(c.webServiceURL, "https://") { c.authParams = config.AuthParams - c.tlsOptions = config.TlsOptions + c.tlsOptions = config.TLSOptions mapAuthParams := make(map[string]string) err := json.Unmarshal([]byte(c.authParams), &mapAuthParams) @@ -158,13 +158,14 @@ func (c *client) getTLSConfig() (*tls.Config, error) { } func (c *client) endpoint(componentPath string, parts ...string) string { - return path.Join(makeHttpPath(c.apiVersion, componentPath), endpoint(parts...)) + return path.Join(makeHTTPPath(c.apiVersion, componentPath), endpoint(parts...)) } // get is used to do a GET request against an endpoint // and deserialize the response into an interface -func (c *client) getWithQueryParams(endpoint string, obj interface{}, params map[string]string, decode bool) ([]byte, error) { +func (c *client) getWithQueryParams(endpoint string, obj interface{}, params map[string]string, + decode bool) ([]byte, error) { req, err := c.newRequest(http.MethodGet, endpoint) if err != nil { @@ -186,7 +187,7 @@ func (c *client) getWithQueryParams(endpoint string, obj interface{}, params map defer safeRespClose(resp) if obj != nil { - if err := decodeJsonBody(resp, &obj); err != nil { + if err := decodeJSONBody(resp, &obj); err != nil { return nil, err } } else if !decode { @@ -205,8 +206,8 @@ func (c *client) get(endpoint string, obj interface{}) error { return err } -func (c *client) put(endpoint string, in, obj interface{}) error { - return c.putWithQueryParams(endpoint, in, obj, nil) +func (c *client) put(endpoint string, in interface{}) error { + return c.putWithQueryParams(endpoint, in, nil, nil) } func (c *client) putWithQueryParams(endpoint string, in, obj interface{}, params map[string]string) error { @@ -231,7 +232,7 @@ func (c *client) putWithQueryParams(endpoint string, in, obj interface{}, params defer safeRespClose(resp) if obj != nil { - if err := decodeJsonBody(resp, &obj); err != nil { + if err := decodeJSONBody(resp, &obj); err != nil { return err } } @@ -239,8 +240,8 @@ func (c *client) putWithQueryParams(endpoint string, in, obj interface{}, params return nil } -func (c *client) delete(endpoint string, obj interface{}) error { - return c.deleteWithQueryParams(endpoint, obj, nil) +func (c *client) delete(endpoint string) error { + return c.deleteWithQueryParams(endpoint, nil, nil) } func (c *client) deleteWithQueryParams(endpoint string, obj interface{}, params map[string]string) error { @@ -264,7 +265,7 @@ func (c *client) deleteWithQueryParams(endpoint string, obj interface{}, params defer safeRespClose(resp) if obj != nil { - if err := decodeJsonBody(resp, &obj); err != nil { + if err := decodeJSONBody(resp, &obj); err != nil { return err } } @@ -272,50 +273,42 @@ func (c *client) deleteWithQueryParams(endpoint string, obj interface{}, params return nil } -func (c *client) post(endpoint string, in, obj interface{}) error { +func (c *client) post(endpoint string, in interface{}) error { req, err := c.newRequest(http.MethodPost, endpoint) if err != nil { return err } req.obj = in + + // nolint resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return err } defer safeRespClose(resp) - if obj != nil { - if err := decodeJsonBody(resp, &obj); err != nil { - return err - } - } + return nil } -func (c *client) putWithMultiPart(endpoint string, in, obj interface{}, body io.Reader, contentType string) error { +func (c *client) putWithMultiPart(endpoint string, body io.Reader, contentType string) error { req, err := c.newRequest(http.MethodPut, endpoint) if err != nil { return err } - req.obj = in req.body = body req.contentType = contentType + // nolint resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return err } defer safeRespClose(resp) - if obj != nil { - if err := decodeJsonBody(resp, &obj); err != nil { - return err - } - } - return nil } -func (c *client) postWithMultiPart(endpoint string, in, obj interface{}, body io.Reader, contentType string) error { +func (c *client) postWithMultiPart(endpoint string, in interface{}, body io.Reader, contentType string) error { req, err := c.newRequest(http.MethodPost, endpoint) if err != nil { return err @@ -324,18 +317,13 @@ func (c *client) postWithMultiPart(endpoint string, in, obj interface{}, body io req.body = body req.contentType = contentType + // nolint resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return err } defer safeRespClose(resp) - if obj != nil { - if err := decodeJsonBody(resp, &obj); err != nil { - return err - } - } - return nil } @@ -354,7 +342,7 @@ func (r *request) toHTTP() (*http.Request, error) { // add a request body if there is one if r.body == nil && r.obj != nil { - body, err := encodeJsonBody(r.obj) + body, err := encodeJSONBody(r.obj) if err != nil { return nil, err } @@ -373,7 +361,7 @@ func (r *request) toHTTP() (*http.Request, error) { } func (c *client) newRequest(method, path string) (*request, error) { - base, _ := url.Parse(c.webServiceUrl) + base, _ := url.Parse(c.webServiceURL) u, err := url.Parse(path) if err != nil { return nil, err @@ -424,8 +412,8 @@ func (c *client) doRequest(r *request) (*http.Response, error) { return hc.Do(req) } -// decodeJsonBody is used to JSON encode a body -func encodeJsonBody(obj interface{}) (io.Reader, error) { +// encodeJSONBody is used to JSON encode a body +func encodeJSONBody(obj interface{}) (io.Reader, error) { buf := bytes.NewBuffer(nil) enc := json.NewEncoder(buf) if err := enc.Encode(obj); err != nil { @@ -434,18 +422,17 @@ func encodeJsonBody(obj interface{}) (io.Reader, error) { return buf, nil } -// decodeJsonBody is used to JSON decode a body -func decodeJsonBody(resp *http.Response, out interface{}) error { +// decodeJSONBody is used to JSON decode a body +func decodeJSONBody(resp *http.Response, out interface{}) error { dec := json.NewDecoder(resp.Body) return dec.Decode(out) } -// safeRespClose is used to close a respone body +// safeRespClose is used to close a response body func safeRespClose(resp *http.Response) { if resp != nil { - if err := resp.Body.Close(); err != nil { - // ignore error since it is closing a response body - } + // ignore error since it is closing a response body + _ = resp.Body.Close() } } diff --git a/pulsaradmin/pkg/pulsar/api_version.go b/pulsaradmin/pkg/pulsar/api_version.go index 07c52972a4..70963d9434 100644 --- a/pulsaradmin/pkg/pulsar/api_version.go +++ b/pulsaradmin/pkg/pulsar/api_version.go @@ -17,17 +17,17 @@ package pulsar -type ApiVersion int +type APIVersion int const ( - V1 ApiVersion = iota + V1 APIVersion = iota V2 V3 ) -const DefaultApiVersion = "v2" +const DefaultAPIVersion = "v2" -func (v ApiVersion) String() string { +func (v APIVersion) String() string { switch v { case V1: return "" @@ -37,5 +37,5 @@ func (v ApiVersion) String() string { return "v3" } - return DefaultApiVersion + return DefaultAPIVersion } diff --git a/pulsaradmin/pkg/pulsar/api_version_test.go b/pulsaradmin/pkg/pulsar/api_version_test.go index 2a0c3a88f7..adcddccfa3 100644 --- a/pulsaradmin/pkg/pulsar/api_version_test.go +++ b/pulsaradmin/pkg/pulsar/api_version_test.go @@ -18,12 +18,13 @@ package pulsar import ( - "github.com/stretchr/testify/assert" "testing" + + "github.com/stretchr/testify/assert" ) func TestApiVersion_String(t *testing.T) { assert.Equal(t, "", V1.String()) assert.Equal(t, "v2", V2.String()) assert.Equal(t, "v3", V3.String()) -} \ No newline at end of file +} diff --git a/pulsaradmin/pkg/pulsar/backlog_quota.go b/pulsaradmin/pkg/pulsar/backlog_quota.go index 8d83b78717..c3ece4cf38 100644 --- a/pulsaradmin/pkg/pulsar/backlog_quota.go +++ b/pulsaradmin/pkg/pulsar/backlog_quota.go @@ -36,7 +36,7 @@ type BacklogQuotaType string const DestinationStorage BacklogQuotaType = "destination_storage" const ( - ProducerRequestHold RetentionPolicy = "producer_request_hold" - ProducerException RetentionPolicy = "producer_exception" + ProducerRequestHold RetentionPolicy = "producer_request_hold" + ProducerException RetentionPolicy = "producer_exception" ConsumerBacklogEviction RetentionPolicy = "consumer_backlog_eviction" ) diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index fcd3cd700b..9b486825c0 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -61,17 +61,17 @@ func (c *clusters) Get(name string) (ClusterData, error) { func (c *clusters) Create(cdata ClusterData) error { endpoint := c.client.endpoint(c.basePath, cdata.Name) - return c.client.put(endpoint, &cdata, nil) + return c.client.put(endpoint, &cdata) } func (c *clusters) Delete(name string) error { endpoint := c.client.endpoint(c.basePath, name) - return c.client.delete(endpoint, nil) + return c.client.delete(endpoint) } func (c *clusters) Update(cdata ClusterData) error { endpoint := c.client.endpoint(c.basePath, cdata.Name) - return c.client.post(endpoint, &cdata, nil) + return c.client.post(endpoint, &cdata) } func (c *clusters) GetPeerClusters(name string) ([]string, error) { @@ -83,12 +83,12 @@ func (c *clusters) GetPeerClusters(name string) ([]string, error) { func (c *clusters) UpdatePeerClusters(cluster string, peerClusters []string) error { endpoint := c.client.endpoint(c.basePath, cluster, "peers") - return c.client.post(endpoint, peerClusters, nil) + return c.client.post(endpoint, peerClusters) } func (c *clusters) CreateFailureDomain(data FailureDomainData) error { endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) - return c.client.post(endpoint, &data, nil) + return c.client.post(endpoint, &data) } func (c *clusters) GetFailureDomain(clusterName string, domainName string) (FailureDomainData, error) { @@ -107,9 +107,9 @@ func (c *clusters) ListFailureDomains(clusterName string) (FailureDomainMap, err func (c *clusters) DeleteFailureDomain(data FailureDomainData) error { endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) - return c.client.delete(endpoint, nil) + return c.client.delete(endpoint) } func (c *clusters) UpdateFailureDomain(data FailureDomainData) error { endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) - return c.client.post(endpoint, &data, nil) + return c.client.post(endpoint, &data) } diff --git a/pulsaradmin/pkg/pulsar/connectorDefinition.go b/pulsaradmin/pkg/pulsar/connectorDefinition.go index 4585e355da..71b3aad290 100644 --- a/pulsaradmin/pkg/pulsar/connectorDefinition.go +++ b/pulsaradmin/pkg/pulsar/connectorDefinition.go @@ -19,17 +19,17 @@ package pulsar // Basic information about a Pulsar connector type ConnectorDefinition struct { - // The name of the connector type - Name string `json:"name"` + // The name of the connector type + Name string `json:"name"` - // Description to be used for user help - Description string `json:"description"` + // Description to be used for user help + Description string `json:"description"` - // The class name for the connector source implementation - //

If not defined, it will be assumed this connector cannot act as a data source - SourceClass string `json:"sourceClass"` + // The class name for the connector source implementation + //

If not defined, it will be assumed this connector cannot act as a data source + SourceClass string `json:"sourceClass"` - // The class name for the connector sink implementation - //

If not defined, it will be assumed this connector cannot act as a data sink - SinkClass string `json:"sinkClass"` + // The class name for the connector sink implementation + //

If not defined, it will be assumed this connector cannot act as a data sink + SinkClass string `json:"sinkClass"` } diff --git a/pulsaradmin/pkg/pulsar/consumerConfig.go b/pulsaradmin/pkg/pulsar/consumerConfig.go index 6d334aff46..5cf84971c2 100644 --- a/pulsaradmin/pkg/pulsar/consumerConfig.go +++ b/pulsaradmin/pkg/pulsar/consumerConfig.go @@ -18,8 +18,8 @@ package pulsar type ConsumerConfig struct { - SchemaType string - SerdeClassName string - IsRegexPattern bool - ReceiverQueueSize int + SchemaType string + SerdeClassName string + IsRegexPattern bool + ReceiverQueueSize int } diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index 18cbf29949..ea40c19d2d 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -29,52 +29,51 @@ type ClusterData struct { // FunctionData information for a Pulsar Function type FunctionData struct { - FQFN string `json:"fqfn"` - Tenant string `json:"tenant"` - Namespace string `json:"namespace"` - FuncName string `json:"functionName"` - InstanceID string `json:"instance_id"` - ClassName string `json:"className"` - Jar string `json:"jarFile"` - Py string `json:"pyFile"` - Go string `json:"goFile"` - Inputs string `json:"inputs"` - TopicsPattern string `json:"topicsPattern"` - Output string `json:"output"` - LogTopic string `json:"logTopic"` - SchemaType string `json:"schemaType"` - CustomSerDeInputs string `json:"customSerdeInputString"` - CustomSchemaInput string `json:"customSchemaInputString"` - OutputSerDeClassName string `json:"outputSerdeClassName"` - FunctionConfigFile string `json:"fnConfigFile"` - ProcessingGuarantees string `json:"processingGuarantees"` - UserConfig string `json:"userConfigString"` - RetainOrdering bool `json:"retainOrdering"` - SubsName string `json:"subsName"` - Parallelism int `json:"parallelism"` - CPU float64 `json:"cpu"` + UpdateAuthData bool `json:"updateAuthData"` + RetainOrdering bool `json:"retainOrdering"` + Watch bool `json:"watch"` + AutoAck bool `json:"autoAck"` + Parallelism int `json:"parallelism"` + WindowLengthCount int `json:"windowLengthCount"` + SlidingIntervalCount int `json:"slidingIntervalCount"` + MaxMessageRetries int `json:"maxMessageRetries"` + + TimeoutMs int64 `json:"timeoutMs"` + SlidingIntervalDurationMs int64 `json:"slidingIntervalDurationMs"` + WindowLengthDurationMs int64 `json:"windowLengthDurationMs"` RAM int64 `json:"ram"` Disk int64 `json:"disk"` - WindowLengthCount int `json:"windowLengthCount"` - WindowLengthDurationMs int64 `json:"windowLengthDurationMs"` - SlidingIntervalCount int `json:"slidingIntervalCount"` - SlidingIntervalDurationMs int64 `json:"slidingIntervalDurationMs"` - AutoAck bool `json:"autoAck"` - TimeoutMs int64 `json:"timeoutMs"` - MaxMessageRetries int `json:"maxMessageRetries"` + CPU float64 `json:"cpu"` + SubsName string `json:"subsName"` DeadLetterTopic string `json:"deadLetterTopic"` + Key string `json:"key"` + State string `json:"state"` + TriggerValue string `json:"triggerValue"` + TriggerFile string `json:"triggerFile"` + Topic string `json:"topic"` - Key string `json:"key"` - Watch bool `json:"watch"` - State string `json:"state"` - TriggerValue string `json:"triggerValue"` - TriggerFile string `json:"triggerFile"` - Topic string `json:"topic"` - - UpdateAuthData bool `json:"updateAuthData"` - - FuncConf *FunctionConfig `json:"-"` - UserCodeFile string `json:"-"` + UserCodeFile string `json:"-"` + FQFN string `json:"fqfn"` + Tenant string `json:"tenant"` + Namespace string `json:"namespace"` + FuncName string `json:"functionName"` + InstanceID string `json:"instance_id"` + ClassName string `json:"className"` + Jar string `json:"jarFile"` + Py string `json:"pyFile"` + Go string `json:"goFile"` + Inputs string `json:"inputs"` + TopicsPattern string `json:"topicsPattern"` + Output string `json:"output"` + LogTopic string `json:"logTopic"` + SchemaType string `json:"schemaType"` + CustomSerDeInputs string `json:"customSerdeInputString"` + CustomSchemaInput string `json:"customSchemaInputString"` + OutputSerDeClassName string `json:"outputSerdeClassName"` + FunctionConfigFile string `json:"fnConfigFile"` + ProcessingGuarantees string `json:"processingGuarantees"` + UserConfig string `json:"userConfigString"` + FuncConf *FunctionConfig `json:"-"` } // Failure Domain information @@ -118,32 +117,30 @@ type SourceData struct { } type SinkData struct { - Tenant string `json:"tenant"` - Namespace string `json:"namespace"` - Name string `json:"name"` - SinkType string `json:"sinkType"` - Inputs string `json:"inputs"` - TopicsPattern string `json:"topicsPattern"` - SubsName string `json:"subsName"` - CustomSerdeInputString string `json:"customSerdeInputString"` - CustomSchemaInputString string `json:"customSchemaInputString"` - ProcessingGuarantees string `json:"processingGuarantees"` - RetainOrdering bool `json:"retainOrdering"` - Parallelism int `json:"parallelism"` - Archive string `json:"archive"` - ClassName string `json:"className"` - SinkConfigFile string `json:"sinkConfigFile"` - CPU float64 `json:"cpu"` - RAM int64 `json:"ram"` - Disk int64 `json:"disk"` - SinkConfigString string `json:"sinkConfigString"` - AutoAck bool `json:"autoAck"` - TimeoutMs int64 `json:"timeoutMs"` - - SinkConf *SinkConfig `json:"-"` - InstanceID string `json:"instanceId"` - - UpdateAuthData bool `json:"updateAuthData"` + UpdateAuthData bool `json:"updateAuthData"` + RetainOrdering bool `json:"retainOrdering"` + AutoAck bool `json:"autoAck"` + Parallelism int `json:"parallelism"` + RAM int64 `json:"ram"` + Disk int64 `json:"disk"` + TimeoutMs int64 `json:"timeoutMs"` + CPU float64 `json:"cpu"` + Tenant string `json:"tenant"` + Namespace string `json:"namespace"` + Name string `json:"name"` + SinkType string `json:"sinkType"` + Inputs string `json:"inputs"` + TopicsPattern string `json:"topicsPattern"` + SubsName string `json:"subsName"` + CustomSerdeInputString string `json:"customSerdeInputString"` + CustomSchemaInputString string `json:"customSchemaInputString"` + ProcessingGuarantees string `json:"processingGuarantees"` + Archive string `json:"archive"` + ClassName string `json:"className"` + SinkConfigFile string `json:"sinkConfigFile"` + SinkConfigString string `json:"sinkConfigString"` + InstanceID string `json:"instanceId"` + SinkConf *SinkConfig `json:"-"` } // Topic data @@ -159,24 +156,25 @@ type ManagedLedgerInfo struct { TerminatedPosition PositionInfo `json:"terminatedPosition"` Cursors map[string]CursorInfo `json:"cursors"` } - type NamespacesData struct { + +type NamespacesData struct { + Enable bool `json:"enable"` + Unload bool `json:"unload"` NumBundles int `json:"numBundles"` - Clusters []string `json:"clusters"` - ClusterIds string `json:"clusterIds"` + BookkeeperEnsemble int `json:"bookkeeperEnsemble"` + BookkeeperWriteQuorum int `json:"bookkeeperWriteQuorum"` MessageTTL int `json:"messageTTL"` + BookkeeperAckQuorum int `json:"bookkeeperAckQuorum"` + ManagedLedgerMaxMarkDeleteRate float64 `json:"managedLedgerMaxMarkDeleteRate"` + ClusterIds string `json:"clusterIds"` RetentionTimeStr string `json:"retentionTimeStr"` LimitStr string `json:"limitStr"` PolicyStr string `json:"policyStr"` AntiAffinityGroup string `json:"antiAffinityGroup"` Tenant string `json:"tenant"` Cluster string `json:"cluster"` - BookkeeperEnsemble int `json:"bookkeeperEnsemble"` - BookkeeperWriteQuorum int `json:"bookkeeperWriteQuorum"` - BookkeeperAckQuorum int `json:"bookkeeperAckQuorum"` - ManagedLedgerMaxMarkDeleteRate float64 `json:"managedLedgerMaxMarkDeleteRate"` - Enable bool `json:"enable"` Bundle string `json:"bundle"` - Unload bool `json:"unload"` + Clusters []string `json:"clusters"` } type TopicStats struct { @@ -193,7 +191,7 @@ type TopicStats struct { } type PublisherStats struct { - ProducerId int64 `json:"producerId"` + ProducerID int64 `json:"producerId"` MsgRateIn float64 `json:"msgRateIn"` MsgThroughputIn float64 `json:"msgThroughputIn"` AverageMsgSize float64 `json:"averageMsgSize"` @@ -201,39 +199,39 @@ type PublisherStats struct { } type SubscriptionStats struct { + BlockedSubscriptionOnUnackedMsgs bool `json:"blockedSubscriptionOnUnackedMsgs"` + IsReplicated bool `json:"isReplicated"` MsgRateOut float64 `json:"msgRateOut"` MsgThroughputOut float64 `json:"msgThroughputOut"` MsgRateRedeliver float64 `json:"msgRateRedeliver"` MsgRateExpired float64 `json:"msgRateExpired"` MsgBacklog int64 `json:"msgBacklog"` - BlockedSubscriptionOnUnackedMsgs bool `json:"blockedSubscriptionOnUnackedMsgs"` MsgDelayed int64 `json:"msgDelayed"` - unAckedMessages int64 `json:"unackedMessages"` + UnAckedMessages int64 `json:"unackedMessages"` SubType string `json:"type"` ActiveConsumerName string `json:"activeConsumerName"` Consumers []ConsumerStats `json:"consumers"` - IsReplicated bool `json:"isReplicated"` } type ConsumerStats struct { - ConsumerName string `json:"consumerName"` + BlockedConsumerOnUnAckedMsgs bool `json:"blockedConsumerOnUnackedMsgs"` + AvailablePermits int `json:"availablePermits"` + UnAckedMessages int `json:"unackedMessages"` MsgRateOut float64 `json:"msgRateOut"` MsgThroughputOut float64 `json:"msgThroughputOut"` MsgRateRedeliver float64 `json:"msgRateRedeliver"` - AvailablePermits int `json:"availablePermits"` - UnAckedMessages int `json:"unackedMessages"` - BlockedConsumerOnUnAckedMsgs bool `json:"blockedConsumerOnUnackedMsgs"` + ConsumerName string `json:"consumerName"` Metadata map[string]string `json:"metadata"` } type ReplicatorStats struct { + Connected bool `json:"connected"` MsgRateIn float64 `json:"msgRateIn"` MsgRateOut float64 `json:"msgRateOut"` MsgThroughputIn float64 `json:"msgThroughputIn"` MsgThroughputOut float64 `json:"msgThroughputOut"` MsgRateExpired float64 `json:"msgRateExpired"` ReplicationBacklog int64 `json:"replicationBacklog"` - Connected bool `json:"connected"` ReplicationDelayInSeconds int64 `json:"replicationDelayInSeconds"` InboundConnection string `json:"inboundConnection"` InboundConnectedSince string `json:"inboundConnectedSince"` @@ -242,6 +240,8 @@ type ReplicatorStats struct { } type PersistentTopicInternalStats struct { + WaitingCursorsCount int `json:"waitingCursorsCount"` + PendingAddEntriesCount int `json:"pendingAddEntriesCount"` EntriesAddedCounter int64 `json:"entriesAddedCounter"` NumberOfEntries int64 `json:"numberOfEntries"` TotalSize int64 `json:"totalSize"` @@ -249,8 +249,6 @@ type PersistentTopicInternalStats struct { CurrentLedgerSize int64 `json:"currentLedgerSize"` LastLedgerCreatedTimestamp string `json:"lastLedgerCreatedTimestamp"` LastLedgerCreationFailureTimestamp string `json:"lastLedgerCreationFailureTimestamp"` - WaitingCursorsCount int `json:"waitingCursorsCount"` - PendingAddEntriesCount int `json:"pendingAddEntriesCount"` LastConfirmedEntry string `json:"lastConfirmedEntry"` State string `json:"state"` Ledgers []LedgerInfo `json:"ledgers"` @@ -258,7 +256,7 @@ type PersistentTopicInternalStats struct { } type LedgerInfo struct { - LedgerId int64 `json:"ledgerId"` + LedgerID int64 `json:"ledgerId"` Entries int64 `json:"entries"` Size int64 `json:"size"` Timestamp int64 `json:"timestamp"` @@ -268,21 +266,21 @@ type CursorInfo struct { Version int `json:"version"` CreationDate string `json:"creationDate"` ModificationDate string `json:"modificationDate"` - CursorsLedgerId int64 `json:"cursorsLedgerId"` + CursorsLedgerID int64 `json:"cursorsLedgerId"` MarkDelete PositionInfo `json:"markDelete"` IndividualDeletedMessages []MessageRangeInfo `json:"individualDeletedMessages"` Properties map[string]int64 } type PositionInfo struct { - LedgerId int64 `json:"ledgerId"` - EntryId int64 `json:"entryId"` + LedgerID int64 `json:"ledgerId"` + EntryID int64 `json:"entryId"` } type MessageRangeInfo struct { - From PositionInfo `json:"from"` - To PositionInfo `json:"to"` - Offloaded bool `json:"offloaded"` + From PositionInfo `json:"from"` + To PositionInfo `json:"to"` + Offloaded bool `json:"offloaded"` } type CursorStats struct { @@ -327,8 +325,8 @@ type SchemaData struct { } type LookupData struct { - BrokerUrl string `json:"brokerUrl"` - BrokerUrlTls string `json:"brokerUrlTls"` - HttpUrl string `json:"httpUrl"` - HttpUrlTls string `json:"httpUrlTls"` + BrokerURL string `json:"brokerUrl"` + BrokerURLTLS string `json:"brokerUrlTls"` + HTTPURL string `json:"httpUrl"` + HTTPURLTLS string `json:"httpUrlTls"` } diff --git a/pulsaradmin/pkg/pulsar/descriptions.go b/pulsaradmin/pkg/pulsar/descriptions.go index 991108ed9d..9a5a3e16f1 100644 --- a/pulsaradmin/pkg/pulsar/descriptions.go +++ b/pulsaradmin/pkg/pulsar/descriptions.go @@ -20,7 +20,7 @@ package pulsar import "strings" var SPACES = " " -var USED_FOR = "USED FOR:" +var USEDFOR = "USED FOR:" var PERMISSION = "REQUIRED PERMISSION:" var EXAMPLES = "EXAMPLES:" var OUTPUT = "OUTPUT:" @@ -43,7 +43,7 @@ type Output struct { } func (desc *LongDescription) ToString() string { - return USED_FOR + "\n" + + return USEDFOR + "\n" + SPACES + desc.CommandUsedFor + "\n\n" + PERMISSION + "\n" + SPACES + desc.CommandPermission + "\n\n" + diff --git a/pulsaradmin/pkg/pulsar/descriptions_test.go b/pulsaradmin/pkg/pulsar/descriptions_test.go index 7a0d65f00e..23ba0d82c9 100644 --- a/pulsaradmin/pkg/pulsar/descriptions_test.go +++ b/pulsaradmin/pkg/pulsar/descriptions_test.go @@ -18,8 +18,9 @@ package pulsar import ( - "github.com/stretchr/testify/assert" "testing" + + "github.com/stretchr/testify/assert" ) func TestLongDescription_exampleToString(t *testing.T) { diff --git a/pulsaradmin/pkg/pulsar/errors.go b/pulsaradmin/pkg/pulsar/errors.go index ae3a317cbb..10ba9503d9 100644 --- a/pulsaradmin/pkg/pulsar/errors.go +++ b/pulsaradmin/pkg/pulsar/errors.go @@ -22,8 +22,8 @@ import "fmt" const unknownErrorReason = "Unknown pulsar error" type Error struct { - Reason string `json:"reason"` - Code int + Reason string `json:"reason"` + Code int } func (e Error) Error() string { diff --git a/pulsaradmin/pkg/pulsar/functionConfg.go b/pulsaradmin/pkg/pulsar/functionConfg.go index 13eb356e36..0590ec1194 100644 --- a/pulsaradmin/pkg/pulsar/functionConfg.go +++ b/pulsaradmin/pkg/pulsar/functionConfg.go @@ -18,60 +18,60 @@ package pulsar const ( - JavaRuntime = "JAVA" - PythonRuntime = "PYTHON" - GoRuntime = "GO" + JavaRuntime = "JAVA" + PythonRuntime = "PYTHON" + GoRuntime = "GO" ) type FunctionConfig struct { - // Any flags that you want to pass to the runtime. - // note that in thread mode, these flags will have no impact - RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` + TimeoutMs *int64 `json:"timeoutMs" yaml:"timeoutMs"` + TopicsPattern *string `json:"topicsPattern" yaml:"topicsPattern"` + // Whether the subscriptions the functions created/used should be deleted when the functions is deleted + CleanupSubscription bool `json:"cleanupSubscription" yaml:"cleanupSubscription"` + RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` + AutoAck bool `json:"autoAck" yaml:"autoAck"` + Parallelism int `json:"parallelism" yaml:"parallelism"` + MaxMessageRetries int `json:"maxMessageRetries" yaml:"maxMessageRetries"` - Tenant string `json:"tenant" yaml:"tenant"` - Namespace string `json:"namespace" yaml:"namespace"` - Name string `json:"name" yaml:"name"` - ClassName string `json:"className" yaml:"className"` + Output string `json:"output" yaml:"output"` - Inputs []string `json:"inputs" yaml:"inputs"` - CustomSerdeInputs map[string]string `json:"customSerdeInputs" yaml:"customSerdeInputs"` - TopicsPattern *string `json:"topicsPattern" yaml:"topicsPattern"` - CustomSchemaInputs map[string]string `json:"customSchemaInputs" yaml:"customSchemaInputs"` + OutputSerdeClassName string `json:"outputSerdeClassName" yaml:"outputSerdeClassName"` + LogTopic string `json:"logTopic" yaml:"logTopic"` + ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` - // A generalized way of specifying inputs - InputSpecs map[string]ConsumerConfig `json:"inputSpecs" yaml:"inputSpecs"` + // Represents either a builtin schema type (eg: 'avro', 'json', etc) or the class name for a Schema implementation + OutputSchemaType string `json:"outputSchemaType" yaml:"outputSchemaType"` - Output string `json:"output" yaml:"output"` + Runtime string `json:"runtime" yaml:"runtime"` + DeadLetterTopic string `json:"deadLetterTopic" yaml:"deadLetterTopic"` + SubName string `json:"subName" yaml:"subName"` + FQFN string `json:"fqfn" yaml:"fqfn"` + Jar string `json:"jar" yaml:"jar"` + Py string `json:"py" yaml:"py"` + Go string `json:"go" yaml:"go"` + // Any flags that you want to pass to the runtime. + // note that in thread mode, these flags will have no impact + RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` - // Represents either a builtin schema type (eg: 'avro', 'json', ect) or the class name for a Schema implementation - OutputSchemaType string `json:"outputSchemaType" yaml:"outputSchemaType"` + Tenant string `json:"tenant" yaml:"tenant"` + Namespace string `json:"namespace" yaml:"namespace"` + Name string `json:"name" yaml:"name"` + ClassName string `json:"className" yaml:"className"` - OutputSerdeClassName string `json:"outputSerdeClassName" yaml:"outputSerdeClassName"` - LogTopic string `json:"logTopic" yaml:"logTopic"` - ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` - RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` - UserConfig map[string]interface{} `json:"userConfig" yaml:"userConfig"` + Resources *Resources `json:"resources" yaml:"resources"` + WindowConfig *WindowConfig `json:"windowConfig" yaml:"windowConfig"` + Inputs []string `json:"inputs" yaml:"inputs"` + UserConfig map[string]interface{} `json:"userConfig" yaml:"userConfig"` + CustomSerdeInputs map[string]string `json:"customSerdeInputs" yaml:"customSerdeInputs"` + CustomSchemaInputs map[string]string `json:"customSchemaInputs" yaml:"customSchemaInputs"` - // This is a map of secretName(aka how the secret is going to be - // accessed in the function via context) to an object that - // encapsulates how the secret is fetched by the underlying - // secrets provider. The type of an value here can be found by the - // SecretProviderConfigurator.getSecretObjectType() method. - Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` + // A generalized way of specifying inputs + InputSpecs map[string]ConsumerConfig `json:"inputSpecs" yaml:"inputSpecs"` - Runtime string `json:"runtime" yaml:"runtime"` - AutoAck bool `json:"autoAck" yaml:"autoAck"` - MaxMessageRetries int `json:"maxMessageRetries" yaml:"maxMessageRetries"` - DeadLetterTopic string `json:"deadLetterTopic" yaml:"deadLetterTopic"` - SubName string `json:"subName" yaml:"subName"` - Parallelism int `json:"parallelism" yaml:"parallelism"` - Resources *Resources `json:"resources" yaml:"resources"` - FQFN string `json:"fqfn" yaml:"fqfn"` - WindowConfig *WindowConfig `json:"windowConfig" yaml:"windowConfig"` - TimeoutMs *int64 `json:"timeoutMs" yaml:"timeoutMs"` - Jar string `json:"jar" yaml:"jar"` - Py string `json:"py" yaml:"py"` - Go string `json:"go" yaml:"go"` - // Whether the subscriptions the functions created/used should be deleted when the functions is deleted - CleanupSubscription bool `json:"cleanupSubscription" yaml:"cleanupSubscription"` + // This is a map of secretName(aka how the secret is going to be + // accessed in the function via context) to an object that + // encapsulates how the secret is fetched by the underlying + // secrets provider. The type of an value here can be found by the + // SecretProviderConfigurator.getSecretObjectType() method. + Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` } diff --git a/pulsaradmin/pkg/pulsar/function_status.go b/pulsaradmin/pkg/pulsar/function_status.go index 1c699beb20..b560eec620 100644 --- a/pulsaradmin/pkg/pulsar/function_status.go +++ b/pulsaradmin/pkg/pulsar/function_status.go @@ -24,7 +24,7 @@ type FunctionStatus struct { } type FunctionInstanceStatus struct { - InstanceId int `json:"instanceId"` + InstanceID int `json:"instanceId"` Status FunctionInstanceStatusData `json:"status"` } @@ -40,14 +40,10 @@ type FunctionInstanceStatusData struct { LatestSystemExceptions []ExceptionInformation `json:"latestSystemExceptions"` AverageLatency float64 `json:"averageLatency"` LastInvocationTime int64 `json:"lastInvocationTime"` - WorkerId string `json:"workerId"` + WorkerID string `json:"workerId"` } type ExceptionInformation struct { ExceptionString string `json:"exceptionString"` TimestampMs int64 `json:"timestampMs"` } - -func (fs *FunctionStatus) AddInstance(functionInstanceStatus FunctionInstanceStatus) { - fs.Instances = append(fs.Instances, functionInstanceStatus) -} diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index b02e29d10f..514fd8e630 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -40,9 +40,9 @@ type Functions interface { // // @param functionConfig // the function configuration object - // @param pkgUrl + // @param pkgURL // url from which pkg can be downloaded - CreateFuncWithUrl(data *FunctionConfig, pkgUrl string) error + CreateFuncWithURL(data *FunctionConfig, pkgURL string) error // Stop all function instances StopFunction(tenant, namespace, name string) error @@ -104,7 +104,7 @@ type Functions interface { // eg: // File: file:/dir/fileName.jar // Http: http://www.repo.com/fileName.jar - UpdateFunctionWithUrl(functionConfig *FunctionConfig, pkgUrl string, updateOptions *UpdateOptions) error + UpdateFunctionWithURL(functionConfig *FunctionConfig, pkgURL string, updateOptions *UpdateOptions) error } type functions struct { @@ -184,7 +184,7 @@ func (f *functions) CreateFunc(funcConf *FunctionConfig, fileName string) error } contentType := multiPartWriter.FormDataContentType() - err = f.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + err = f.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) if err != nil { return err } @@ -192,7 +192,7 @@ func (f *functions) CreateFunc(funcConf *FunctionConfig, fileName string) error return nil } -func (f *functions) CreateFuncWithUrl(funcConf *FunctionConfig, pkgUrl string) error { +func (f *functions) CreateFuncWithURL(funcConf *FunctionConfig, pkgURL string) error { endpoint := f.client.endpoint(f.basePath, funcConf.Tenant, funcConf.Namespace, funcConf.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -204,7 +204,7 @@ func (f *functions) CreateFuncWithUrl(funcConf *FunctionConfig, pkgUrl string) e return err } - _, err = textWriter.Write([]byte(pkgUrl)) + _, err = textWriter.Write([]byte(pkgURL)) if err != nil { return err } @@ -229,7 +229,7 @@ func (f *functions) CreateFuncWithUrl(funcConf *FunctionConfig, pkgUrl string) e } contentType := multiPartWriter.FormDataContentType() - err = f.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + err = f.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) if err != nil { return err } @@ -238,44 +238,44 @@ func (f *functions) CreateFuncWithUrl(funcConf *FunctionConfig, pkgUrl string) e } func (f *functions) StopFunction(tenant, namespace, name string) error { - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - return f.client.post(endpoint+"/stop", "", nil) + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) + return f.client.post(endpoint+"/stop", "") } -func (f *functions) StopFunctionWithID(tenant, namespace, name string, instanceID int) error { - id := fmt.Sprintf("%d", instanceID) - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) +func (f *functions) StopFunctionWithID(tenant, namespace, name string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) - return f.client.post(endpoint+"/stop", "", nil) + return f.client.post(endpoint+"/stop", "") } func (f *functions) DeleteFunction(tenant, namespace, name string) error { endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - return f.client.delete(endpoint, nil) + return f.client.delete(endpoint) } func (f *functions) StartFunction(tenant, namespace, name string) error { endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - return f.client.post(endpoint+"/start", "", nil) + return f.client.post(endpoint+"/start", "") } -func (f *functions) StartFunctionWithID(tenant, namespace, name string, instanceID int) error { +func (f *functions) StartFunctionWithID(tenant, namespace, name string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) - return f.client.post(endpoint+"/start", "", nil) + return f.client.post(endpoint+"/start", "") } func (f *functions) RestartFunction(tenant, namespace, name string) error { endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - return f.client.post(endpoint+"/restart", "", nil) + return f.client.post(endpoint+"/restart", "") } -func (f *functions) RestartFunctionWithID(tenant, namespace, name string, instanceID int) error { +func (f *functions) RestartFunctionWithID(tenant, namespace, name string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) - return f.client.post(endpoint+"/restart", "", nil) + return f.client.post(endpoint+"/restart", "") } func (f *functions) GetFunctions(tenant, namespace string) ([]string, error) { @@ -285,14 +285,15 @@ func (f *functions) GetFunctions(tenant, namespace string) ([]string, error) { return functions, err } -func (f *functions) GetFunction(tenant, namespace, name string) (FunctionConfig, error) { +func (f *functions) GetFunction(tenant, namespace, name string) (FunctionConfig, error) { var functionConfig FunctionConfig endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) err := f.client.get(endpoint, &functionConfig) return functionConfig, err } -func (f *functions) UpdateFunction(functionConfig *FunctionConfig, fileName string, updateOptions *UpdateOptions) error { +func (f *functions) UpdateFunction(functionConfig *FunctionConfig, fileName string, + updateOptions *UpdateOptions) error { endpoint := f.client.endpoint(f.basePath, functionConfig.Tenant, functionConfig.Namespace, functionConfig.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -359,7 +360,7 @@ func (f *functions) UpdateFunction(functionConfig *FunctionConfig, fileName stri } contentType := multiPartWriter.FormDataContentType() - err = f.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + err = f.client.putWithMultiPart(endpoint, bodyBuf, contentType) if err != nil { return err } @@ -367,7 +368,8 @@ func (f *functions) UpdateFunction(functionConfig *FunctionConfig, fileName stri return nil } -func (f *functions) UpdateFunctionWithUrl(functionConfig *FunctionConfig, pkgUrl string, updateOptions *UpdateOptions) error { +func (f *functions) UpdateFunctionWithURL(functionConfig *FunctionConfig, pkgURL string, + updateOptions *UpdateOptions) error { endpoint := f.client.endpoint(f.basePath, functionConfig.Tenant, functionConfig.Namespace, functionConfig.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -379,7 +381,7 @@ func (f *functions) UpdateFunctionWithUrl(functionConfig *FunctionConfig, pkgUrl return err } - _, err = textWriter.Write([]byte(pkgUrl)) + _, err = textWriter.Write([]byte(pkgURL)) if err != nil { return err } @@ -423,7 +425,7 @@ func (f *functions) UpdateFunctionWithUrl(functionConfig *FunctionConfig, pkgUrl } contentType := multiPartWriter.FormDataContentType() - err = f.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + err = f.client.putWithMultiPart(endpoint, bodyBuf, contentType) if err != nil { return err } @@ -438,7 +440,8 @@ func (f *functions) GetFunctionStatus(tenant, namespace, name string) (FunctionS return functionStatus, err } -func (f *functions) GetFunctionStatusWithInstanceID(tenant, namespace, name string, instanceID int) (FunctionInstanceStatusData, error) { +func (f *functions) GetFunctionStatusWithInstanceID(tenant, namespace, name string, + instanceID int) (FunctionInstanceStatusData, error) { var functionInstanceStatusData FunctionInstanceStatusData id := fmt.Sprintf("%d", instanceID) endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) @@ -453,7 +456,8 @@ func (f *functions) GetFunctionStats(tenant, namespace, name string) (FunctionSt return functionStats, err } -func (f *functions) GetFunctionStatsWithInstanceID(tenant, namespace, name string, instanceID int) (FunctionInstanceStatsData, error) { +func (f *functions) GetFunctionStatsWithInstanceID(tenant, namespace, name string, + instanceID int) (FunctionInstanceStatsData, error) { var functionInstanceStatsData FunctionInstanceStatsData id := fmt.Sprintf("%d", instanceID) endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) @@ -461,7 +465,7 @@ func (f *functions) GetFunctionStatsWithInstanceID(tenant, namespace, name strin return functionInstanceStatsData, err } -func (f *functions)GetFunctionState(tenant, namespace, name, key string) (FunctionState, error) { +func (f *functions) GetFunctionState(tenant, namespace, name, key string) (FunctionState, error) { var functionState FunctionState endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, "state", key) err := f.client.get(endpoint, &functionState) @@ -501,7 +505,7 @@ func (f *functions) PutFunctionState(tenant, namespace, name string, state Funct contentType := multiPartWriter.FormDataContentType() - err = f.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) + err = f.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) if err != nil { return err @@ -570,7 +574,7 @@ func (f *functions) TriggerFunction(tenant, namespace, name, topic, triggerValue contentType := multiPartWriter.FormDataContentType() var str string - err := f.client.postWithMultiPart(endpoint, &str, nil, bodyBuf, contentType) + err := f.client.postWithMultiPart(endpoint, &str, bodyBuf, contentType) if err != nil { return "", err } diff --git a/pulsaradmin/pkg/pulsar/functions_stats.go b/pulsaradmin/pkg/pulsar/functions_stats.go index 3f644388b5..3e6ecc4efe 100644 --- a/pulsaradmin/pkg/pulsar/functions_stats.go +++ b/pulsaradmin/pkg/pulsar/functions_stats.go @@ -46,7 +46,7 @@ type FunctionStats struct { type FunctionInstanceStats struct { FunctionInstanceStatsDataBase - InstanceId int64 `json:"instanceId"` + InstanceID int64 `json:"instanceId"` Metrics FunctionInstanceStatsData `json:"metrics"` } @@ -129,14 +129,14 @@ func (fs *FunctionStats) CalculateOverall() *FunctionStats { // calculate average from sum if nonNullInstances > 0 { - fs.AvgProcessLatency = fs.AvgProcessLatency / float64(nonNullInstances) + fs.AvgProcessLatency /= float64(nonNullInstances) } else { fs.AvgProcessLatency = 0 } // calculate 1min average from sum if nonNullInstancesOneMin > 0 { - fs.OneMin.AvgProcessLatency = fs.OneMin.AvgProcessLatency / float64(nonNullInstancesOneMin) + fs.OneMin.AvgProcessLatency /= float64(nonNullInstancesOneMin) } else { fs.AvgProcessLatency = 0 } diff --git a/pulsaradmin/pkg/pulsar/message_id.go b/pulsaradmin/pkg/pulsar/message_id.go index a86a830c5d..42e3fc7f3c 100644 --- a/pulsaradmin/pkg/pulsar/message_id.go +++ b/pulsaradmin/pkg/pulsar/message_id.go @@ -17,8 +17,8 @@ package pulsar -type MessageId struct { - LedgerId int64 - EntryId int64 - PartitionedIndex int +type MessageID struct { + LedgerID int64 `json:"ledgerId"` + EntryID int64 `json:"entryId"` + PartitionedIndex int `json:"partitionedIndex"` } diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index f3c42a7d49..d1e29a2f58 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -142,16 +142,20 @@ type Namespaces interface { // Clear backlog for all topics on a namespace ClearNamespaceBacklog(namespace NameSpaceName) error - // Set replicator-message-dispatch-rate (Replicators under this namespace can dispatch this many messages per second) + // Set replicator-message-dispatch-rate (Replicators under this namespace + // can dispatch this many messages per second) SetReplicatorDispatchRate(namespace NameSpaceName, rate DispatchRate) error - // Get replicator-message-dispatch-rate (Replicators under this namespace can dispatch this many messages per second) + // Get replicator-message-dispatch-rate (Replicators under this namespace + // can dispatch this many messages per second) GetReplicatorDispatchRate(namespace NameSpaceName) (DispatchRate, error) - // Set subscription-message-dispatch-rate (subscriptions under this namespace can dispatch this many messages per second) + // Set subscription-message-dispatch-rate (subscriptions under this namespace + // can dispatch this many messages per second) SetSubscriptionDispatchRate(namespace NameSpaceName, rate DispatchRate) error - // Get subscription-message-dispatch-rate (subscriptions under this namespace can dispatch this many messages per second) + // Get subscription-message-dispatch-rate (subscriptions under this namespace + // can dispatch this many messages per second) GetSubscriptionDispatchRate(namespace NameSpaceName) (DispatchRate, error) // Set namespace-subscribe-rate (topics under this namespace will limit by subscribeRate) @@ -218,7 +222,7 @@ func (n *namespaces) CreateNsWithPolices(namespace string, policies Policies) er return err } endpoint := n.client.endpoint(n.basePath, ns.String()) - return n.client.put(endpoint, &policies, nil) + return n.client.put(endpoint, &policies) } func (n *namespaces) CreateNsWithBundlesData(namespace string, bundleData *BundlesData) error { @@ -230,7 +234,7 @@ func (n *namespaces) CreateNsWithBundlesData(namespace string, bundleData *Bundl polices := new(Policies) polices.Bundles = bundleData - return n.client.put(endpoint, &polices, nil) + return n.client.put(endpoint, &polices) } func (n *namespaces) CreateNamespace(namespace string) error { @@ -239,7 +243,7 @@ func (n *namespaces) CreateNamespace(namespace string) error { return err } endpoint := n.client.endpoint(n.basePath, ns.String()) - return n.client.put(endpoint, nil, nil) + return n.client.put(endpoint, nil) } func (n *namespaces) DeleteNamespace(namespace string) error { @@ -248,7 +252,7 @@ func (n *namespaces) DeleteNamespace(namespace string) error { return err } endpoint := n.client.endpoint(n.basePath, ns.String()) - return n.client.delete(endpoint, nil) + return n.client.delete(endpoint) } func (n *namespaces) DeleteNamespaceBundle(namespace string, bundleRange string) error { @@ -257,7 +261,7 @@ func (n *namespaces) DeleteNamespaceBundle(namespace string, bundleRange string) return err } endpoint := n.client.endpoint(n.basePath, ns.String(), bundleRange) - return n.client.delete(endpoint, nil) + return n.client.delete(endpoint) } func (n *namespaces) GetNamespaceMessageTTL(namespace string) (int, error) { @@ -278,7 +282,7 @@ func (n *namespaces) SetNamespaceMessageTTL(namespace string, ttlInSeconds int) } endpoint := n.client.endpoint(n.basePath, nsName.String(), "messageTTL") - return n.client.post(endpoint, &ttlInSeconds, nil) + return n.client.post(endpoint, &ttlInSeconds) } func (n *namespaces) SetRetention(namespace string, policy RetentionPolicies) error { @@ -287,7 +291,7 @@ func (n *namespaces) SetRetention(namespace string, policy RetentionPolicies) er return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), "retention") - return n.client.post(endpoint, &policy, nil) + return n.client.post(endpoint, &policy) } func (n *namespaces) GetRetention(namespace string) (*RetentionPolicies, error) { @@ -318,7 +322,7 @@ func (n *namespaces) SetBacklogQuota(namespace string, backlogQuota BacklogQuota return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuota") - return n.client.post(endpoint, &backlogQuota, nil) + return n.client.post(endpoint, &backlogQuota) } func (n *namespaces) RemoveBacklogQuota(namespace string) error { @@ -350,7 +354,7 @@ func (n *namespaces) SetNamespaceReplicationClusters(namespace string, clusterId return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), "replication") - return n.client.post(endpoint, &clusterIds, nil) + return n.client.post(endpoint, &clusterIds) } func (n *namespaces) SetNamespaceAntiAffinityGroup(namespace string, namespaceAntiAffinityGroup string) error { @@ -359,7 +363,7 @@ func (n *namespaces) SetNamespaceAntiAffinityGroup(namespace string, namespaceAn return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), "antiAffinity") - return n.client.post(endpoint, namespaceAntiAffinityGroup, nil) + return n.client.post(endpoint, namespaceAntiAffinityGroup) } func (n *namespaces) GetAntiAffinityNamespaces(tenant, cluster, namespaceAntiAffinityGroup string) ([]string, error) { @@ -388,7 +392,7 @@ func (n *namespaces) DeleteNamespaceAntiAffinityGroup(namespace string) error { return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), "antiAffinity") - return n.client.delete(endpoint, nil) + return n.client.delete(endpoint) } func (n *namespaces) SetDeduplicationStatus(namespace string, enableDeduplication bool) error { @@ -397,7 +401,7 @@ func (n *namespaces) SetDeduplicationStatus(namespace string, enableDeduplicatio return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), "deduplication") - return n.client.post(endpoint, enableDeduplication, nil) + return n.client.post(endpoint, enableDeduplication) } func (n *namespaces) SetPersistence(namespace string, persistence PersistencePolicies) error { @@ -406,7 +410,7 @@ func (n *namespaces) SetPersistence(namespace string, persistence PersistencePol return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence") - return n.client.post(endpoint, &persistence, nil) + return n.client.post(endpoint, &persistence) } func (n *namespaces) SetBookieAffinityGroup(namespace string, bookieAffinityGroup BookieAffinityGroupData) error { @@ -415,7 +419,7 @@ func (n *namespaces) SetBookieAffinityGroup(namespace string, bookieAffinityGrou return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence", "bookieAffinity") - return n.client.post(endpoint, &bookieAffinityGroup, nil) + return n.client.post(endpoint, &bookieAffinityGroup) } func (n *namespaces) DeleteBookieAffinityGroup(namespace string) error { @@ -424,7 +428,7 @@ func (n *namespaces) DeleteBookieAffinityGroup(namespace string) error { return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence", "bookieAffinity") - return n.client.delete(endpoint, nil) + return n.client.delete(endpoint) } func (n *namespaces) GetBookieAffinityGroup(namespace string) (*BookieAffinityGroupData, error) { @@ -455,7 +459,7 @@ func (n *namespaces) Unload(namespace string) error { return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), "unload") - return n.client.put(endpoint, "", nil) + return n.client.put(endpoint, "") } func (n *namespaces) UnloadNamespaceBundle(namespace, bundle string) error { @@ -464,7 +468,7 @@ func (n *namespaces) UnloadNamespaceBundle(namespace, bundle string) error { return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), bundle, "unload") - return n.client.put(endpoint, "", nil) + return n.client.put(endpoint, "") } func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitBundles bool) error { @@ -481,47 +485,47 @@ func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitB func (n *namespaces) SetSubscriptionAuthMode(namespace NameSpaceName, mode SubscriptionAuthMode) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionAuthMode") - return n.client.post(endpoint, mode.String(), nil) + return n.client.post(endpoint, mode.String()) } func (n *namespaces) SetEncryptionRequiredStatus(namespace NameSpaceName, encrypt bool) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "encryptionRequired") - return n.client.post(endpoint, strconv.FormatBool(encrypt), nil) + return n.client.post(endpoint, strconv.FormatBool(encrypt)) } func (n *namespaces) UnsubscribeNamespace(namespace NameSpaceName, sName string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "unsubscribe", url.QueryEscape(sName)) - return n.client.post(endpoint, "", nil) + return n.client.post(endpoint, "") } func (n *namespaces) UnsubscribeNamespaceBundle(namespace NameSpaceName, bundle, sName string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "unsubscribe", url.QueryEscape(sName)) - return n.client.post(endpoint, "", nil) + return n.client.post(endpoint, "") } func (n *namespaces) ClearNamespaceBundleBacklogForSubscription(namespace NameSpaceName, bundle, sName string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog", url.QueryEscape(sName)) - return n.client.post(endpoint, "", nil) + return n.client.post(endpoint, "") } func (n *namespaces) ClearNamespaceBundleBacklog(namespace NameSpaceName, bundle string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog") - return n.client.post(endpoint, "", nil) + return n.client.post(endpoint, "") } func (n *namespaces) ClearNamespaceBacklogForSubscription(namespace NameSpaceName, sName string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "clearBacklog", url.QueryEscape(sName)) - return n.client.post(endpoint, "", nil) + return n.client.post(endpoint, "") } func (n *namespaces) ClearNamespaceBacklog(namespace NameSpaceName) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "clearBacklog") - return n.client.post(endpoint, "", nil) + return n.client.post(endpoint, "") } func (n *namespaces) SetReplicatorDispatchRate(namespace NameSpaceName, rate DispatchRate) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "replicatorDispatchRate") - return n.client.post(endpoint, rate, nil) + return n.client.post(endpoint, rate) } func (n *namespaces) GetReplicatorDispatchRate(namespace NameSpaceName) (DispatchRate, error) { @@ -533,7 +537,7 @@ func (n *namespaces) GetReplicatorDispatchRate(namespace NameSpaceName) (Dispatc func (n *namespaces) SetSubscriptionDispatchRate(namespace NameSpaceName, rate DispatchRate) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionDispatchRate") - return n.client.post(endpoint, rate, nil) + return n.client.post(endpoint, rate) } func (n *namespaces) GetSubscriptionDispatchRate(namespace NameSpaceName) (DispatchRate, error) { @@ -545,7 +549,7 @@ func (n *namespaces) GetSubscriptionDispatchRate(namespace NameSpaceName) (Dispa func (n *namespaces) SetSubscribeRate(namespace NameSpaceName, rate SubscribeRate) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscribeRate") - return n.client.post(endpoint, rate, nil) + return n.client.post(endpoint, rate) } func (n *namespaces) GetSubscribeRate(namespace NameSpaceName) (SubscribeRate, error) { @@ -557,7 +561,7 @@ func (n *namespaces) GetSubscribeRate(namespace NameSpaceName) (SubscribeRate, e func (n *namespaces) SetDispatchRate(namespace NameSpaceName, rate DispatchRate) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "dispatchRate") - return n.client.post(endpoint, rate, nil) + return n.client.post(endpoint, rate) } func (n *namespaces) GetDispatchRate(namespace NameSpaceName) (DispatchRate, error) { diff --git a/pulsaradmin/pkg/pulsar/namespace_name.go b/pulsaradmin/pkg/pulsar/namespace_name.go index 3e1c2e5b49..ac57214611 100644 --- a/pulsaradmin/pkg/pulsar/namespace_name.go +++ b/pulsaradmin/pkg/pulsar/namespace_name.go @@ -19,9 +19,10 @@ package pulsar import ( "fmt" - "github.com/pkg/errors" "regexp" "strings" + + "github.com/pkg/errors" ) type NameSpaceName struct { @@ -30,14 +31,14 @@ type NameSpaceName struct { } func GetNameSpaceName(tenant, namespace string) (*NameSpaceName, error) { - return GetNamespaceName(fmt.Sprintf("%s/%s", tenant,namespace)) + return GetNamespaceName(fmt.Sprintf("%s/%s", tenant, namespace)) } func GetNamespaceName(completeName string) (*NameSpaceName, error) { var n NameSpaceName if completeName == "" { - return nil, errors.New("The namespace complete name is empty.") + return nil, errors.New("the namespace complete name is empty") } parts := strings.Split(completeName, "/") diff --git a/pulsaradmin/pkg/pulsar/namespace_name_test.go b/pulsaradmin/pkg/pulsar/namespace_name_test.go index fc7b04f839..4b5294afee 100644 --- a/pulsaradmin/pkg/pulsar/namespace_name_test.go +++ b/pulsaradmin/pkg/pulsar/namespace_name_test.go @@ -18,8 +18,9 @@ package pulsar import ( - "github.com/stretchr/testify/assert" "testing" + + "github.com/stretchr/testify/assert" ) func TestGetNamespaceName(t *testing.T) { @@ -29,7 +30,7 @@ func TestGetNamespaceName(t *testing.T) { empty, err := GetNamespaceName("") assert.NotNil(t, err) - assert.Equal(t, "The namespace complete name is empty.", err.Error()) + assert.Equal(t, "the namespace complete name is empty", err.Error()) assert.Nil(t, empty) empty, err = GetNamespaceName("/") diff --git a/pulsaradmin/pkg/pulsar/persistence_policies.go b/pulsaradmin/pkg/pulsar/persistence_policies.go index a8739c1e8d..3c0a2229ef 100644 --- a/pulsaradmin/pkg/pulsar/persistence_policies.go +++ b/pulsaradmin/pkg/pulsar/persistence_policies.go @@ -24,16 +24,8 @@ type PersistencePolicies struct { ManagedLedgerMaxMarkDeleteRate float64 `json:"managedLedgerMaxMarkDeleteRate"` } -func NewDefaultPersistencePolicies() *PersistencePolicies { - return &PersistencePolicies{ - BookkeeperEnsemble: 2, - BookkeeperWriteQuorum: 2, - BookkeeperAckQuorum: 2, - ManagedLedgerMaxMarkDeleteRate: 0.0, - } -} - -func NewPersistencePolicies(bookkeeperEnsemble, bookkeeperWriteQuorum, bookkeeperAckQuorum int, managedLedgerMaxMarkDeleteRate float64) PersistencePolicies { +func NewPersistencePolicies(bookkeeperEnsemble, bookkeeperWriteQuorum, bookkeeperAckQuorum int, + managedLedgerMaxMarkDeleteRate float64) PersistencePolicies { return PersistencePolicies{ BookkeeperEnsemble: bookkeeperEnsemble, BookkeeperWriteQuorum: bookkeeperWriteQuorum, @@ -46,10 +38,3 @@ type BookieAffinityGroupData struct { BookkeeperAffinityGroupPrimary string `json:"bookkeeperAffinityGroupPrimary"` BookkeeperAffinityGroupSecondary string `json:"bookkeeperAffinityGroupSecondary"` } - -func NewBookieAffinityGroupData(bookkeeperAffinityGroupPrimary, bookkeeperAffinityGroupSecondary string) BookieAffinityGroupData { - return BookieAffinityGroupData{ - BookkeeperAffinityGroupPrimary: bookkeeperAffinityGroupPrimary, - BookkeeperAffinityGroupSecondary: bookkeeperAffinityGroupSecondary, - } -} diff --git a/pulsaradmin/pkg/pulsar/policies.go b/pulsaradmin/pkg/pulsar/policies.go index f664335144..392f416c16 100644 --- a/pulsaradmin/pkg/pulsar/policies.go +++ b/pulsaradmin/pkg/pulsar/policies.go @@ -23,54 +23,53 @@ const ( ) type Policies struct { - AuthPolicies AuthPolicies `json:"auth_policies"` - ReplicationClusters []string `json:"replication_clusters"` - Bundles *BundlesData `json:"bundles"` - BacklogQuotaMap map[BacklogQuotaType]BacklogQuota `json:"backlog_quota_map"` - TopicDispatchRate map[string]DispatchRate `json:"topicDispatchRate"` - SubscriptionDispatchRate map[string]DispatchRate `json:"subscriptionDispatchRate"` - ReplicatorDispatchRate map[string]DispatchRate `json:"replicatorDispatchRate"` - ClusterSubscribeRate map[string]SubscribeRate `json:"clusterSubscribeRate"` - Persistence *PersistencePolicies `json:"persistence"` - DeduplicationEnabled bool `json:"deduplicationEnabled"` - LatencyStatsSampleRate map[string]int `json:"latency_stats_sample_rate"` - MessageTtlInSeconds int `json:"message_ttl_in_seconds"` - RetentionPolicies *RetentionPolicies `json:"retention_policies"` - Deleted bool `json:"deleted"` - AntiAffinityGroup string `json:"antiAffinityGroup"` - EncryptionRequired bool `json:"encryption_required"` - SubscriptionAuthMode SubscriptionAuthMode `json:"subscription_auth_mode"` - MaxProducersPerTopic int `json:"max_producers_per_topic"` - MaxConsumersPerTopic int `json:"max_consumers_per_topic"` - MaxConsumersPerSubscription int `json:"max_consumers_per_subscription"` - CompactionThreshold int64 `json:"compaction_threshold"` - OffloadThreshold int64 `json:"offload_threshold"` - OffloadDeletionLagMs int64 `json:"offload_deletion_lag_ms"` - SchemaAutoUpdateCompatibilityStrategy SchemaAutoUpdateCompatibilityStrategy `json:"schema_auto_update_compatibility_strategy"` - SchemaValidationEnforced bool `json:"schema_validation_enforced"` + Bundles *BundlesData `json:"bundles"` + Persistence *PersistencePolicies `json:"persistence"` + RetentionPolicies *RetentionPolicies `json:"retention_policies"` + SchemaValidationEnforced bool `json:"schema_validation_enforced"` + DeduplicationEnabled bool `json:"deduplicationEnabled"` + Deleted bool `json:"deleted"` + EncryptionRequired bool `json:"encryption_required"` + MessageTTLInSeconds int `json:"message_ttl_in_seconds"` + MaxProducersPerTopic int `json:"max_producers_per_topic"` + MaxConsumersPerTopic int `json:"max_consumers_per_topic"` + MaxConsumersPerSubscription int `json:"max_consumers_per_subscription"` + CompactionThreshold int64 `json:"compaction_threshold"` + OffloadThreshold int64 `json:"offload_threshold"` + OffloadDeletionLagMs int64 `json:"offload_deletion_lag_ms"` + AntiAffinityGroup string `json:"antiAffinityGroup"` + ReplicationClusters []string `json:"replication_clusters"` + LatencyStatsSampleRate map[string]int `json:"latency_stats_sample_rate"` + BacklogQuotaMap map[BacklogQuotaType]BacklogQuota `json:"backlog_quota_map"` + TopicDispatchRate map[string]DispatchRate `json:"topicDispatchRate"` + SubscriptionDispatchRate map[string]DispatchRate `json:"subscriptionDispatchRate"` + ReplicatorDispatchRate map[string]DispatchRate `json:"replicatorDispatchRate"` + ClusterSubscribeRate map[string]SubscribeRate `json:"clusterSubscribeRate"` + SchemaCompatibilityStrategy SchemaCompatibilityStrategy `json:"schema_auto_update_compatibility_strategy"` + AuthPolicies AuthPolicies `json:"auth_policies"` + SubscriptionAuthMode SubscriptionAuthMode `json:"subscription_auth_mode"` } func NewDefaultPolicies() *Policies { return &Policies{ - AuthPolicies: *NewAuthPolicies(), - ReplicationClusters: make([]string, 0, 10), - BacklogQuotaMap: make(map[BacklogQuotaType]BacklogQuota), - TopicDispatchRate: make(map[string]DispatchRate), - SubscriptionDispatchRate: make(map[string]DispatchRate), - ReplicatorDispatchRate: make(map[string]DispatchRate), - ClusterSubscribeRate: make(map[string]SubscribeRate), - LatencyStatsSampleRate: make(map[string]int), - MessageTtlInSeconds: 0, - Deleted: false, - EncryptionRequired: false, - SubscriptionAuthMode: None, - MaxProducersPerTopic: 0, - MaxConsumersPerSubscription: 0, - MaxConsumersPerTopic: 0, - CompactionThreshold: 0, - OffloadThreshold: -1, - SchemaAutoUpdateCompatibilityStrategy: Full, - SchemaValidationEnforced: false, + AuthPolicies: *NewAuthPolicies(), + ReplicationClusters: make([]string, 0, 10), + BacklogQuotaMap: make(map[BacklogQuotaType]BacklogQuota), + TopicDispatchRate: make(map[string]DispatchRate), + SubscriptionDispatchRate: make(map[string]DispatchRate), + ReplicatorDispatchRate: make(map[string]DispatchRate), + ClusterSubscribeRate: make(map[string]SubscribeRate), + LatencyStatsSampleRate: make(map[string]int), + MessageTTLInSeconds: 0, + Deleted: false, + EncryptionRequired: false, + SubscriptionAuthMode: None, + MaxProducersPerTopic: 0, + MaxConsumersPerSubscription: 0, + MaxConsumersPerTopic: 0, + CompactionThreshold: 0, + OffloadThreshold: -1, + SchemaCompatibilityStrategy: Full, + SchemaValidationEnforced: false, } } - diff --git a/pulsaradmin/pkg/pulsar/resources.go b/pulsaradmin/pkg/pulsar/resources.go index f1fd1431b7..bfe03375a8 100644 --- a/pulsaradmin/pkg/pulsar/resources.go +++ b/pulsaradmin/pkg/pulsar/resources.go @@ -18,39 +18,20 @@ package pulsar type Resources struct { - CPU float64 `json:"cpu"` - Disk int64 `json:"disk"` - Ram int64 `json:"ram"` + CPU float64 `json:"cpu"` + Disk int64 `json:"disk"` + RAM int64 `json:"ram"` } func NewDefaultResources() *Resources { - resources:= &Resources{ - //Default cpu is 1 core - CPU: 1, - // Default memory is 1GB - Disk: 1073741824, - // Default disk is 10GB - Ram: 10737418240, - } - - return resources -} - -func NewResources(resources *Resources) *Resources { - if resources == nil { - return NewDefaultResources() - } - - if resources.CPU == 0 { - resources.CPU = 1 - } - - if resources.Disk == 0 { - resources.Disk = 1073741824 - } - - if resources.Ram == 0 { - resources.Ram = 10737418240 - } - return resources + resources := &Resources{ + //Default cpu is 1 core + CPU: 1, + // Default memory is 1GB + Disk: 1073741824, + // Default disk is 10GB + RAM: 10737418240, + } + + return resources } diff --git a/pulsaradmin/pkg/pulsar/schema.go b/pulsaradmin/pkg/pulsar/schema.go index 494acea332..880992e739 100644 --- a/pulsaradmin/pkg/pulsar/schema.go +++ b/pulsaradmin/pkg/pulsar/schema.go @@ -117,7 +117,7 @@ func (s *schemas) DeleteSchema(topic string) error { fmt.Println(endpoint) - return s.client.delete(endpoint, nil) + return s.client.delete(endpoint) } func (s *schemas) CreateSchemaByPayload(topic string, schemaPayload PostSchemaPayload) error { @@ -129,5 +129,5 @@ func (s *schemas) CreateSchemaByPayload(topic string, schemaPayload PostSchemaPa endpoint := s.client.endpoint(s.basePath, topicName.tenant, topicName.namespace, topicName.GetEncodedTopic(), "schema") - return s.client.post(endpoint, &schemaPayload, nil) + return s.client.post(endpoint, &schemaPayload) } diff --git a/pulsaradmin/pkg/pulsar/schema_strategy.go b/pulsaradmin/pkg/pulsar/schema_strategy.go index 66b4752595..e0f4f2cdbe 100644 --- a/pulsaradmin/pkg/pulsar/schema_strategy.go +++ b/pulsaradmin/pkg/pulsar/schema_strategy.go @@ -17,15 +17,15 @@ package pulsar -type SchemaAutoUpdateCompatibilityStrategy string +type SchemaCompatibilityStrategy string const ( - AutoUpdateDisabled SchemaAutoUpdateCompatibilityStrategy = "AutoUpdateDisabled" - Backward SchemaAutoUpdateCompatibilityStrategy = "Backward" - Forward SchemaAutoUpdateCompatibilityStrategy = "Forward" - Full SchemaAutoUpdateCompatibilityStrategy = "Full" - AlwaysCompatible SchemaAutoUpdateCompatibilityStrategy = "AlwaysCompatible" - BackwardTransitive SchemaAutoUpdateCompatibilityStrategy = "BackwardTransitive" - ForwardTransitive SchemaAutoUpdateCompatibilityStrategy = "ForwardTransitive" - FullTransitive SchemaAutoUpdateCompatibilityStrategy = "FullTransitive" + AutoUpdateDisabled SchemaCompatibilityStrategy = "AutoUpdateDisabled" + Backward SchemaCompatibilityStrategy = "Backward" + Forward SchemaCompatibilityStrategy = "Forward" + Full SchemaCompatibilityStrategy = "Full" + AlwaysCompatible SchemaCompatibilityStrategy = "AlwaysCompatible" + BackwardTransitive SchemaCompatibilityStrategy = "BackwardTransitive" + ForwardTransitive SchemaCompatibilityStrategy = "ForwardTransitive" + FullTransitive SchemaCompatibilityStrategy = "FullTransitive" ) diff --git a/pulsaradmin/pkg/pulsar/sinkConfig.go b/pulsaradmin/pkg/pulsar/sinkConfig.go index bf77168daf..81fee99266 100644 --- a/pulsaradmin/pkg/pulsar/sinkConfig.go +++ b/pulsaradmin/pkg/pulsar/sinkConfig.go @@ -18,34 +18,35 @@ package pulsar type SinkConfig struct { - Tenant string `json:"tenant" yaml:"tenant"` - Namespace string `json:"namespace" yaml:"namespace"` - Name string `json:"name" yaml:"name"` - ClassName string `json:"className" yaml:"className"` + TopicsPattern *string `json:"topicsPattern" yaml:"topicsPattern"` + Resources *Resources `json:"resources" yaml:"resources"` + TimeoutMs *int64 `json:"timeoutMs" yaml:"timeoutMs"` - SourceSubscriptionName string `json:"sourceSubscriptionName" yaml:"sourceSubscriptionName"` - Inputs []string `json:"inputs" yaml:"inputs"` - TopicToSerdeClassName map[string]string `json:"topicToSerdeClassName" yaml:"topicToSerdeClassName"` - TopicsPattern *string `json:"topicsPattern" yaml:"topicsPattern"` - TopicToSchemaType map[string]string `json:"topicToSchemaType" yaml:"topicToSchemaType"` - InputSpecs map[string]ConsumerConfig `json:"inputSpecs" yaml:"inputSpecs"` - Configs map[string]interface{} `json:"configs" yaml:"configs"` + // Whether the subscriptions the functions created/used should be deleted when the functions is deleted + CleanupSubscription bool `json:"cleanupSubscription" yaml:"cleanupSubscription"` - // This is a map of secretName(aka how the secret is going to be - // accessed in the function via context) to an object that - // encapsulates how the secret is fetched by the underlying - // secrets provider. The type of an value here can be found by the - // SecretProviderConfigurator.getSecretObjectType() method. - Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` - Parallelism int `json:"parallelism" yaml:"parallelism"` - ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` - RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` - Resources *Resources `json:"resources" yaml:"resources"` - AutoAck bool `json:"autoAck" yaml:"autoAck"` - TimeoutMs *int64 `json:"timeoutMs" yaml:"timeoutMs"` - Archive string `json:"archive" yaml:"archive"` + RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` + AutoAck bool `json:"autoAck" yaml:"autoAck"` + Parallelism int `json:"parallelism" yaml:"parallelism"` + Tenant string `json:"tenant" yaml:"tenant"` + Namespace string `json:"namespace" yaml:"namespace"` + Name string `json:"name" yaml:"name"` + ClassName string `json:"className" yaml:"className"` - // Whether the subscriptions the functions created/used should be deleted when the functions is deleted - CleanupSubscription bool `json:"cleanupSubscription" yaml:"cleanupSubscription"` - RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` + Archive string `json:"archive" yaml:"archive"` + ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` + SourceSubscriptionName string `json:"sourceSubscriptionName" yaml:"sourceSubscriptionName"` + RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` + Inputs []string `json:"inputs" yaml:"inputs"` + TopicToSerdeClassName map[string]string `json:"topicToSerdeClassName" yaml:"topicToSerdeClassName"` + TopicToSchemaType map[string]string `json:"topicToSchemaType" yaml:"topicToSchemaType"` + InputSpecs map[string]ConsumerConfig `json:"inputSpecs" yaml:"inputSpecs"` + Configs map[string]interface{} `json:"configs" yaml:"configs"` + + // This is a map of secretName(aka how the secret is going to be + // accessed in the function via context) to an object that + // encapsulates how the secret is fetched by the underlying + // secrets provider. The type of an value here can be found by the + // SecretProviderConfigurator.getSecretObjectType() method. + Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` } diff --git a/pulsaradmin/pkg/pulsar/sink_status.go b/pulsaradmin/pkg/pulsar/sink_status.go index 67c79be0f8..68fa27dfb2 100644 --- a/pulsaradmin/pkg/pulsar/sink_status.go +++ b/pulsaradmin/pkg/pulsar/sink_status.go @@ -18,55 +18,50 @@ package pulsar type SinkStatus struct { - // The total number of sink instances that ought to be running - NumInstances int `json:"numInstances"` - // The number of source instances that are actually running - NumRunning int `json:"numRunning"` + // The total number of sink instances that ought to be running + NumInstances int `json:"numInstances"` - Instances []*SinkInstanceStatus `json:"instances"` + // The number of source instances that are actually running + NumRunning int `json:"numRunning"` + + Instances []*SinkInstanceStatus `json:"instances"` } type SinkInstanceStatus struct { - InstanceId int `json:"instanceId"` - - Status SourceInstanceStatusData `json:"status"` + InstanceID int `json:"instanceId"` + Status SourceInstanceStatusData `json:"status"` } type SinkInstanceStatusData struct { - // Is this instance running? - Running bool `json:"running"` + // Is this instance running? + Running bool `json:"running"` - // Do we have any error while running this instance - Err string `json:"error"` + // Do we have any error while running this instance + Err string `json:"error"` - // Number of times this instance has restarted - NumRestarts int64 `json:"numRestarts"` + // Number of times this instance has restarted + NumRestarts int64 `json:"numRestarts"` - // Number of messages read from Pulsar - NumReadFromPulsar int64 `json:"numReadFromPulsar"` + // Number of messages read from Pulsar + NumReadFromPulsar int64 `json:"numReadFromPulsar"` - // Number of times there was a system exception handling messages - NumSystemExceptions int64 `json:"numSystemExceptions"` + // Number of times there was a system exception handling messages + NumSystemExceptions int64 `json:"numSystemExceptions"` - // A list of the most recent system exceptions - LatestSystemExceptions []ExceptionInformation `json:"latestSystemExceptions"` + // A list of the most recent system exceptions + LatestSystemExceptions []ExceptionInformation `json:"latestSystemExceptions"` - // Number of times there was a sink exception - NumSinkExceptions int64 `json:"numSinkExceptions"` + // Number of times there was a sink exception + NumSinkExceptions int64 `json:"numSinkExceptions"` - // A list of the most recent sink exceptions - LatestSinkExceptions []ExceptionInformation `json:"latestSinkExceptions"` + // A list of the most recent sink exceptions + LatestSinkExceptions []ExceptionInformation `json:"latestSinkExceptions"` - // Number of messages written to sink - NumWrittenToSink int64 `json:"numWrittenToSink"` + // Number of messages written to sink + NumWrittenToSink int64 `json:"numWrittenToSink"` - // When was the last time we received a message from Pulsar - LastReceivedTime int64 `json:"lastReceivedTime"` + // When was the last time we received a message from Pulsar + LastReceivedTime int64 `json:"lastReceivedTime"` - WorkerId string `json:"workerId"` + WorkerID string `json:"workerId"` } - -func (ss *SinkStatus)AddInstance(sinkInstanceStatus *SinkInstanceStatus) { - ss.Instances = append(ss.Instances, sinkInstanceStatus) -} - diff --git a/pulsaradmin/pkg/pulsar/sinks.go b/pulsaradmin/pkg/pulsar/sinks.go index 6a31798674..936bd4aceb 100644 --- a/pulsaradmin/pkg/pulsar/sinks.go +++ b/pulsaradmin/pkg/pulsar/sinks.go @@ -18,417 +18,416 @@ package pulsar import ( - `bytes` - `encoding/json` - `fmt` - `io` - `mime/multipart` - `net/textproto` - `os` - `path/filepath` - `strings` + "bytes" + "encoding/json" + "fmt" + "io" + "mime/multipart" + "net/textproto" + "os" + "path/filepath" + "strings" ) type Sinks interface { - // Get the list of all the Pulsar Sinks. - ListSinks(tenant, namespace string) ([]string, error) + // Get the list of all the Pulsar Sinks. + ListSinks(tenant, namespace string) ([]string, error) - // Get the configuration for the specified sink - GetSink(tenant, namespace, Sink string) (SinkConfig, error) + // Get the configuration for the specified sink + GetSink(tenant, namespace, Sink string) (SinkConfig, error) - // Create a new sink - CreateSink(config *SinkConfig, fileName string) error + // Create a new sink + CreateSink(config *SinkConfig, fileName string) error - // Create a new sink by providing url from which fun-pkg can be downloaded. supported url: http/file - CreateSinkWithURL(config *SinkConfig, pkgUrl string) error + // Create a new sink by providing url from which fun-pkg can be downloaded. supported url: http/file + CreateSinkWithURL(config *SinkConfig, pkgURL string) error - // Update the configuration for a sink. - UpdateSink(config *SinkConfig, fileName string, options *UpdateOptions) error + // Update the configuration for a sink. + UpdateSink(config *SinkConfig, fileName string, options *UpdateOptions) error - // Update a sink by providing url from which fun-pkg can be downloaded. supported url: http/file - UpdateSinkWithUrl(config *SinkConfig, pkgUrl string, options *UpdateOptions) error + // Update a sink by providing url from which fun-pkg can be downloaded. supported url: http/file + UpdateSinkWithURL(config *SinkConfig, pkgURL string, options *UpdateOptions) error - // Delete an existing sink - DeleteSink(tenant, namespace, Sink string) error + // Delete an existing sink + DeleteSink(tenant, namespace, Sink string) error - // Gets the current status of a sink. - GetSinkStatus(tenant, namespace, Sink string) (SinkStatus, error) + // Gets the current status of a sink. + GetSinkStatus(tenant, namespace, Sink string) (SinkStatus, error) - // Gets the current status of a sink instance. - GetSinkStatusWithID(tenant, namespace, Sink string, id int) (SinkInstanceStatusData, error) + // Gets the current status of a sink instance. + GetSinkStatusWithID(tenant, namespace, Sink string, id int) (SinkInstanceStatusData, error) - // Restart all sink instances - RestartSink(tenant, namespace, Sink string) error + // Restart all sink instances + RestartSink(tenant, namespace, Sink string) error - // Restart sink instance - RestartSinkWithID(tenant, namespace, Sink string, id int) error + // Restart sink instance + RestartSinkWithID(tenant, namespace, Sink string, id int) error - // Stop all sink instances - StopSink(tenant, namespace, Sink string) error + // Stop all sink instances + StopSink(tenant, namespace, Sink string) error - // Stop sink instance - StopSinkWithID(tenant, namespace, Sink string, id int) error + // Stop sink instance + StopSinkWithID(tenant, namespace, Sink string, id int) error - // Start all sink instances - StartSink(tenant, namespace, Sink string) error + // Start all sink instances + StartSink(tenant, namespace, Sink string) error - // Start sink instance - StartSinkWithID(tenant, namespace, Sink string, id int) error + // Start sink instance + StartSinkWithID(tenant, namespace, Sink string, id int) error - // Fetches a list of supported Pulsar IO sinks currently running in cluster mode - GetBuiltInSinks() ([]*ConnectorDefinition, error) + // Fetches a list of supported Pulsar IO sinks currently running in cluster mode + GetBuiltInSinks() ([]*ConnectorDefinition, error) - // Reload the available built-in connectors, include Source and Sink - ReloadBuiltInSinks() error + // Reload the available built-in connectors, include Source and Sink + ReloadBuiltInSinks() error } type sinks struct { - client *client - basePath string + client *client + basePath string } func (c *client) Sinks() Sinks { - return &sinks{ - client: c, - basePath: "/sinks", - } + return &sinks{ + client: c, + basePath: "/sinks", + } } func (s *sinks) createStringFromField(w *multipart.Writer, value string) (io.Writer, error) { - h := make(textproto.MIMEHeader) - h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) - h.Set("Content-Type", "application/json") - return w.CreatePart(h) + h := make(textproto.MIMEHeader) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) + h.Set("Content-Type", "application/json") + return w.CreatePart(h) } func (s *sinks) createTextFromFiled(w *multipart.Writer, value string) (io.Writer, error) { - h := make(textproto.MIMEHeader) - h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) - h.Set("Content-Type", "text/plain") - return w.CreatePart(h) + h := make(textproto.MIMEHeader) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) + h.Set("Content-Type", "text/plain") + return w.CreatePart(h) } func (s *sinks) ListSinks(tenant, namespace string) ([]string, error) { - var sinks []string - endpoint := s.client.endpoint(s.basePath, tenant, namespace) - err := s.client.get(endpoint, &sinks) - return sinks, err + var sinks []string + endpoint := s.client.endpoint(s.basePath, tenant, namespace) + err := s.client.get(endpoint, &sinks) + return sinks, err } -func (s *sinks) GetSink(tenant, namespace, Sink string) (SinkConfig, error) { - var sinkConfig SinkConfig - endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) - err := s.client.get(endpoint, &sinkConfig) - return sinkConfig, err +func (s *sinks) GetSink(tenant, namespace, sink string) (SinkConfig, error) { + var sinkConfig SinkConfig + endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) + err := s.client.get(endpoint, &sinkConfig) + return sinkConfig, err } func (s *sinks) CreateSink(config *SinkConfig, fileName string) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) - - // buffer to store our request as bytes - bodyBuf := bytes.NewBufferString("") - - multiPartWriter := multipart.NewWriter(bodyBuf) - jsonData, err := json.Marshal(config) - if err != nil { - return err - } - - stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") - if err != nil { - return err - } - - _, err = stringWriter.Write(jsonData) - if err != nil { - return err - } - - if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { - // If the function code is built in, we don't need to submit here - file, err := os.Open(fileName) - if err != nil { - return err - } - defer file.Close() - - part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) - - if err != nil { - return err - } - - // copy the actual file content to the filed's writer - _, err = io.Copy(part, file) - if err != nil { - return err - } - } - - // In here, we completed adding the file and the fields, let's close the multipart writer - // So it writes the ending boundary - if err = multiPartWriter.Close(); err != nil { - return err - } - - contentType := multiPartWriter.FormDataContentType() - err = s.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) - if err != nil { - return err - } - - return nil + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { + // If the function code is built in, we don't need to submit here + file, err := os.Open(fileName) + if err != nil { + return err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) + + if err != nil { + return err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil } -func (s *sinks) CreateSinkWithURL(config *SinkConfig, pkgUrl string) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) - // buffer to store our request as bytes - bodyBuf := bytes.NewBufferString("") - - multiPartWriter := multipart.NewWriter(bodyBuf) - - textWriter, err := s.createTextFromFiled(multiPartWriter, "url") - if err != nil { - return err - } - - _, err = textWriter.Write([]byte(pkgUrl)) - if err != nil { - return err - } - - jsonData, err := json.Marshal(config) - if err != nil { - return err - } - - stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") - if err != nil { - return err - } - - _, err = stringWriter.Write(jsonData) - if err != nil { - return err - } - - if err = multiPartWriter.Close(); err != nil { - return err - } - - contentType := multiPartWriter.FormDataContentType() - err = s.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) - if err != nil { - return err - } - - return nil +func (s *sinks) CreateSinkWithURL(config *SinkConfig, pkgURL string) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + textWriter, err := s.createTextFromFiled(multiPartWriter, "url") + if err != nil { + return err + } + + _, err = textWriter.Write([]byte(pkgURL)) + if err != nil { + return err + } + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil } func (s *sinks) UpdateSink(config *SinkConfig, fileName string, updateOptions *UpdateOptions) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) - // buffer to store our request as bytes - bodyBuf := bytes.NewBufferString("") - - multiPartWriter := multipart.NewWriter(bodyBuf) - - jsonData, err := json.Marshal(config) - if err != nil { - return err - } - - stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") - if err != nil { - return err - } - - _, err = stringWriter.Write(jsonData) - if err != nil { - return err - } - - if updateOptions != nil { - updateData, err := json.Marshal(updateOptions) - if err != nil { - return err - } - - updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") - if err != nil { - return err - } - - _, err = updateStrWriter.Write(updateData) - if err != nil { - return err - } - } - - if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { - // If the function code is built in, we don't need to submit here - file, err := os.Open(fileName) - if err != nil { - return err - } - defer file.Close() - - part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) - - if err != nil { - return err - } - - // copy the actual file content to the filed's writer - _, err = io.Copy(part, file) - if err != nil { - return err - } - } - - // In here, we completed adding the file and the fields, let's close the multipart writer - // So it writes the ending boundary - if err = multiPartWriter.Close(); err != nil { - return err - } - - contentType := multiPartWriter.FormDataContentType() - err = s.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) - if err != nil { - return err - } - - return nil + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if updateOptions != nil { + updateData, err := json.Marshal(updateOptions) + if err != nil { + return err + } + + updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") + if err != nil { + return err + } + + _, err = updateStrWriter.Write(updateData) + if err != nil { + return err + } + } + + if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { + // If the function code is built in, we don't need to submit here + file, err := os.Open(fileName) + if err != nil { + return err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) + + if err != nil { + return err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.putWithMultiPart(endpoint, bodyBuf, contentType) + if err != nil { + return err + } + + return nil } -func (s *sinks) UpdateSinkWithUrl(config *SinkConfig, pkgUrl string, updateOptions *UpdateOptions) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) - // buffer to store our request as bytes - bodyBuf := bytes.NewBufferString("") - - multiPartWriter := multipart.NewWriter(bodyBuf) - - textWriter, err := s.createTextFromFiled(multiPartWriter, "url") - if err != nil { - return err - } - - _, err = textWriter.Write([]byte(pkgUrl)) - if err != nil { - return err - } - - jsonData, err := json.Marshal(config) - if err != nil { - return err - } - - stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") - if err != nil { - return err - } - - _, err = stringWriter.Write(jsonData) - if err != nil { - return err - } - - if updateOptions != nil { - updateData, err := json.Marshal(updateOptions) - if err != nil { - return err - } - - updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") - if err != nil { - return err - } - - _, err = updateStrWriter.Write(updateData) - if err != nil { - return err - } - } - - // In here, we completed adding the file and the fields, let's close the multipart writer - // So it writes the ending boundary - if err = multiPartWriter.Close(); err != nil { - return err - } - - contentType := multiPartWriter.FormDataContentType() - err = s.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) - if err != nil { - return err - } - - return nil +func (s *sinks) UpdateSinkWithURL(config *SinkConfig, pkgURL string, updateOptions *UpdateOptions) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + textWriter, err := s.createTextFromFiled(multiPartWriter, "url") + if err != nil { + return err + } + + _, err = textWriter.Write([]byte(pkgURL)) + if err != nil { + return err + } + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sinkConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if updateOptions != nil { + updateData, err := json.Marshal(updateOptions) + if err != nil { + return err + } + + updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") + if err != nil { + return err + } + + _, err = updateStrWriter.Write(updateData) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.putWithMultiPart(endpoint, bodyBuf, contentType) + if err != nil { + return err + } + + return nil } -func (s *sinks) DeleteSink(tenant, namespace, Sink string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) - return s.client.delete(endpoint, nil) +func (s *sinks) DeleteSink(tenant, namespace, sink string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) + return s.client.delete(endpoint) } -func (s *sinks) GetSinkStatus(tenant, namespace, Sink string) (SinkStatus, error) { - var sinkStatus SinkStatus - endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) - err := s.client.get(endpoint+"/status", &sinkStatus) - return sinkStatus, err +func (s *sinks) GetSinkStatus(tenant, namespace, sink string) (SinkStatus, error) { + var sinkStatus SinkStatus + endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) + err := s.client.get(endpoint+"/status", &sinkStatus) + return sinkStatus, err } -func (s *sinks) GetSinkStatusWithID(tenant, namespace, Sink string, id int) (SinkInstanceStatusData, error) { - var sinkInstanceStatusData SinkInstanceStatusData - instanceID := fmt.Sprintf("%d", id) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink, instanceID) - err := s.client.get(endpoint+"/status", &sinkInstanceStatusData) - return sinkInstanceStatusData, err +func (s *sinks) GetSinkStatusWithID(tenant, namespace, sink string, id int) (SinkInstanceStatusData, error) { + var sinkInstanceStatusData SinkInstanceStatusData + instanceID := fmt.Sprintf("%d", id) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink, instanceID) + err := s.client.get(endpoint+"/status", &sinkInstanceStatusData) + return sinkInstanceStatusData, err } -func (s *sinks) RestartSink(tenant, namespace, Sink string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) - return s.client.post(endpoint+"/restart", "", nil) +func (s *sinks) RestartSink(tenant, namespace, sink string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) + return s.client.post(endpoint+"/restart", "") } -func (s *sinks) RestartSinkWithID(tenant, namespace, Sink string, instanceID int) error { - id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink, id) +func (s *sinks) RestartSinkWithID(tenant, namespace, sink string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink, id) - return s.client.post(endpoint+"/restart", "", nil) + return s.client.post(endpoint+"/restart", "") } -func (s *sinks) StopSink(tenant, namespace, Sink string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) - return s.client.post(endpoint+"/stop", "", nil) +func (s *sinks) StopSink(tenant, namespace, sink string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) + return s.client.post(endpoint+"/stop", "") } -func (s *sinks) StopSinkWithID(tenant, namespace, Sink string, instanceID int) error { - id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink, id) +func (s *sinks) StopSinkWithID(tenant, namespace, sink string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink, id) - return s.client.post(endpoint+"/stop", "", nil) + return s.client.post(endpoint+"/stop", "") } -func (s *sinks) StartSink(tenant, namespace, Sink string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink) - return s.client.post(endpoint+"/start", "", nil) +func (s *sinks) StartSink(tenant, namespace, sink string) error { + endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) + return s.client.post(endpoint+"/start", "") } -func (s *sinks) StartSinkWithID(tenant, namespace, Sink string, instanceID int) error { - id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, Sink, id) +func (s *sinks) StartSinkWithID(tenant, namespace, sink string, instanceID int) error { + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink, id) - return s.client.post(endpoint+"/start", "", nil) + return s.client.post(endpoint+"/start", "") } func (s *sinks) GetBuiltInSinks() ([]*ConnectorDefinition, error) { - var connectorDefinition []*ConnectorDefinition - endpoint := s.client.endpoint(s.basePath, "builtinSinks") - err := s.client.get(endpoint, &connectorDefinition) - return connectorDefinition, err + var connectorDefinition []*ConnectorDefinition + endpoint := s.client.endpoint(s.basePath, "builtinSinks") + err := s.client.get(endpoint, &connectorDefinition) + return connectorDefinition, err } func (s *sinks) ReloadBuiltInSinks() error { - endpoint := s.client.endpoint(s.basePath, "reloadBuiltInSinks") - return s.client.post(endpoint, "", nil) + endpoint := s.client.endpoint(s.basePath, "reloadBuiltInSinks") + return s.client.post(endpoint, "") } - diff --git a/pulsaradmin/pkg/pulsar/sourceConfig.go b/pulsaradmin/pkg/pulsar/sourceConfig.go index 1405a5c587..debbc39c4b 100644 --- a/pulsaradmin/pkg/pulsar/sourceConfig.go +++ b/pulsaradmin/pkg/pulsar/sourceConfig.go @@ -18,28 +18,28 @@ package pulsar type SourceConfig struct { - Tenant string `json:"tenant" yaml:"tenant"` - Namespace string `json:"namespace" yaml:"namespace"` - Name string `json:"name" yaml:"name"` - ClassName string `json:"className" yaml:"className"` + Tenant string `json:"tenant" yaml:"tenant"` + Namespace string `json:"namespace" yaml:"namespace"` + Name string `json:"name" yaml:"name"` + ClassName string `json:"className" yaml:"className"` - TopicName string `json:"topicName" yaml:"topicName"` - SerdeClassName string `json:"serdeClassName" yaml:"serdeClassName"` - SchemaType string `json:"schemaType" yaml:"schemaType"` + TopicName string `json:"topicName" yaml:"topicName"` + SerdeClassName string `json:"serdeClassName" yaml:"serdeClassName"` + SchemaType string `json:"schemaType" yaml:"schemaType"` - Configs map[string]interface{} `json:"configs" yaml:"configs"` + Configs map[string]interface{} `json:"configs" yaml:"configs"` - // This is a map of secretName(aka how the secret is going to be - // accessed in the function via context) to an object that - // encapsulates how the secret is fetched by the underlying - // secrets provider. The type of an value here can be found by the - // SecretProviderConfigurator.getSecretObjectType() method. - Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` + // This is a map of secretName(aka how the secret is going to be + // accessed in the function via context) to an object that + // encapsulates how the secret is fetched by the underlying + // secrets provider. The type of an value here can be found by the + // SecretProviderConfigurator.getSecretObjectType() method. + Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` - Parallelism int `json:"parallelism" yaml:"parallelism"` - ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` - Resources *Resources `json:"resources" yaml:"resources"` - Archive string `json:"archive" yaml:"archive"` - // Any flags that you want to pass to the runtime. - RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` + Parallelism int `json:"parallelism" yaml:"parallelism"` + ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` + Resources *Resources `json:"resources" yaml:"resources"` + Archive string `json:"archive" yaml:"archive"` + // Any flags that you want to pass to the runtime. + RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` } diff --git a/pulsaradmin/pkg/pulsar/source_status.go b/pulsaradmin/pkg/pulsar/source_status.go index 9c3833aee6..1e147980ef 100644 --- a/pulsaradmin/pkg/pulsar/source_status.go +++ b/pulsaradmin/pkg/pulsar/source_status.go @@ -18,42 +18,26 @@ package pulsar type SourceStatus struct { - NumInstances int `json:"numInstances"` - NumRunning int `json:"numRunning"` - - Instances []*SourceInstanceStatus `json:"instances"` + NumInstances int `json:"numInstances"` + NumRunning int `json:"numRunning"` + Instances []*SourceInstanceStatus `json:"instances"` } type SourceInstanceStatus struct { - InstanceId int `json:"instanceId"` - - Status SourceInstanceStatusData `json:"status"` + InstanceID int `json:"instanceId"` + Status SourceInstanceStatusData `json:"status"` } type SourceInstanceStatusData struct { - Running bool `json:"running"` - - Err string `json:"error"` - - NumRestarts int64 `json:"numRestarts"` - - NumReceivedFromSource int64 `json:"numReceivedFromSource"` - - NumSystemExceptions int64 `json:"numSystemExceptions"` - - LatestSystemExceptions []ExceptionInformation `json:"latestSystemExceptions"` - - NumSourceExceptions int64 `json:"numSourceExceptions"` - - LatestSourceExceptions []ExceptionInformation `json:"latestSourceExceptions"` - - NumWritten int64 `json:"numWritten"` - - LastReceivedTime int64 `json:"lastReceivedTime"` - - WorkerId string `json:"workerId"` -} - -func (ss *SourceStatus)AddInstance(sourceInstanceStatus *SourceInstanceStatus) { - ss.Instances = append(ss.Instances, sourceInstanceStatus) + Running bool `json:"running"` + Err string `json:"error"` + NumRestarts int64 `json:"numRestarts"` + NumReceivedFromSource int64 `json:"numReceivedFromSource"` + NumSystemExceptions int64 `json:"numSystemExceptions"` + LatestSystemExceptions []ExceptionInformation `json:"latestSystemExceptions"` + NumSourceExceptions int64 `json:"numSourceExceptions"` + LatestSourceExceptions []ExceptionInformation `json:"latestSourceExceptions"` + NumWritten int64 `json:"numWritten"` + LastReceivedTime int64 `json:"lastReceivedTime"` + WorkerID string `json:"workerId"` } diff --git a/pulsaradmin/pkg/pulsar/sources.go b/pulsaradmin/pkg/pulsar/sources.go index 5d3f9a1d40..ece4b5468a 100644 --- a/pulsaradmin/pkg/pulsar/sources.go +++ b/pulsaradmin/pkg/pulsar/sources.go @@ -18,416 +18,416 @@ package pulsar import ( - `bytes` - `encoding/json` - `fmt` - `io` - `mime/multipart` - `net/textproto` - `os` - `path/filepath` - `strings` + "bytes" + "encoding/json" + "fmt" + "io" + "mime/multipart" + "net/textproto" + "os" + "path/filepath" + "strings" ) type Sources interface { - // Get the list of all the Pulsar Sources. - ListSources(tenant, namespace string) ([]string, error) + // Get the list of all the Pulsar Sources. + ListSources(tenant, namespace string) ([]string, error) - // Get the configuration for the specified source - GetSource(tenant, namespace, source string) (SourceConfig, error) + // Get the configuration for the specified source + GetSource(tenant, namespace, source string) (SourceConfig, error) - // Create a new source - CreateSource(config *SourceConfig, fileName string) error + // Create a new source + CreateSource(config *SourceConfig, fileName string) error - // Create a new source by providing url from which fun-pkg can be downloaded. supported url: http/file - CreateSourceWithURL(config *SourceConfig, pkgUrl string) error + // Create a new source by providing url from which fun-pkg can be downloaded. supported url: http/file + CreateSourceWithURL(config *SourceConfig, pkgURL string) error - // Update the configuration for a source. - UpdateSource(config *SourceConfig, fileName string, options *UpdateOptions) error + // Update the configuration for a source. + UpdateSource(config *SourceConfig, fileName string, options *UpdateOptions) error - // Update a source by providing url from which fun-pkg can be downloaded. supported url: http/file - UpdateSourceWithUrl(config *SourceConfig, pkgUrl string, options *UpdateOptions) error + // Update a source by providing url from which fun-pkg can be downloaded. supported url: http/file + UpdateSourceWithURL(config *SourceConfig, pkgURL string, options *UpdateOptions) error - // Delete an existing source - DeleteSource(tenant, namespace, source string) error + // Delete an existing source + DeleteSource(tenant, namespace, source string) error - // Gets the current status of a source. - GetSourceStatus(tenant, namespace, source string) (SourceStatus, error) + // Gets the current status of a source. + GetSourceStatus(tenant, namespace, source string) (SourceStatus, error) - // Gets the current status of a source instance. - GetSourceStatusWithID(tenant, namespace, source string, id int) (SourceInstanceStatusData, error) + // Gets the current status of a source instance. + GetSourceStatusWithID(tenant, namespace, source string, id int) (SourceInstanceStatusData, error) - // Restart all source instances - RestartSource(tenant, namespace, source string) error + // Restart all source instances + RestartSource(tenant, namespace, source string) error - // Restart source instance - RestartSourceWithID(tenant, namespace, source string, id int) error + // Restart source instance + RestartSourceWithID(tenant, namespace, source string, id int) error - // Stop all source instances - StopSource(tenant, namespace, source string) error + // Stop all source instances + StopSource(tenant, namespace, source string) error - // Stop source instance - StopSourceWithID(tenant, namespace, source string, id int) error + // Stop source instance + StopSourceWithID(tenant, namespace, source string, id int) error - // Start all source instances - StartSource(tenant, namespace, source string) error + // Start all source instances + StartSource(tenant, namespace, source string) error - // Start source instance - StartSourceWithID(tenant, namespace, source string, id int) error + // Start source instance + StartSourceWithID(tenant, namespace, source string, id int) error - // Fetches a list of supported Pulsar IO sources currently running in cluster mode - GetBuiltInSources() ([]*ConnectorDefinition, error) + // Fetches a list of supported Pulsar IO sources currently running in cluster mode + GetBuiltInSources() ([]*ConnectorDefinition, error) - // Reload the available built-in connectors, include Source and Sink - ReloadBuiltInSources() error + // Reload the available built-in connectors, include Source and Sink + ReloadBuiltInSources() error } type sources struct { - client *client - basePath string + client *client + basePath string } func (c *client) Sources() Sources { - return &sources{ - client: c, - basePath: "/sources", - } + return &sources{ + client: c, + basePath: "/sources", + } } func (s *sources) createStringFromField(w *multipart.Writer, value string) (io.Writer, error) { - h := make(textproto.MIMEHeader) - h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) - h.Set("Content-Type", "application/json") - return w.CreatePart(h) + h := make(textproto.MIMEHeader) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) + h.Set("Content-Type", "application/json") + return w.CreatePart(h) } func (s *sources) createTextFromFiled(w *multipart.Writer, value string) (io.Writer, error) { - h := make(textproto.MIMEHeader) - h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) - h.Set("Content-Type", "text/plain") - return w.CreatePart(h) + h := make(textproto.MIMEHeader) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) + h.Set("Content-Type", "text/plain") + return w.CreatePart(h) } func (s *sources) ListSources(tenant, namespace string) ([]string, error) { - var sources []string - endpoint := s.client.endpoint(s.basePath, tenant, namespace) - err := s.client.get(endpoint, &sources) - return sources, err + var sources []string + endpoint := s.client.endpoint(s.basePath, tenant, namespace) + err := s.client.get(endpoint, &sources) + return sources, err } func (s *sources) GetSource(tenant, namespace, source string) (SourceConfig, error) { - var sourceConfig SourceConfig - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - err := s.client.get(endpoint, &sourceConfig) - return sourceConfig, err + var sourceConfig SourceConfig + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + err := s.client.get(endpoint, &sourceConfig) + return sourceConfig, err } func (s *sources) CreateSource(config *SourceConfig, fileName string) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) - - // buffer to store our request as bytes - bodyBuf := bytes.NewBufferString("") - - multiPartWriter := multipart.NewWriter(bodyBuf) - jsonData, err := json.Marshal(config) - if err != nil { - return err - } - - stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") - if err != nil { - return err - } - - _, err = stringWriter.Write(jsonData) - if err != nil { - return err - } - - if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { - // If the function code is built in, we don't need to submit here - file, err := os.Open(fileName) - if err != nil { - return err - } - defer file.Close() - - part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) - - if err != nil { - return err - } - - // copy the actual file content to the filed's writer - _, err = io.Copy(part, file) - if err != nil { - return err - } - } - - // In here, we completed adding the file and the fields, let's close the multipart writer - // So it writes the ending boundary - if err = multiPartWriter.Close(); err != nil { - return err - } - - contentType := multiPartWriter.FormDataContentType() - err = s.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) - if err != nil { - return err - } - - return nil + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { + // If the function code is built in, we don't need to submit here + file, err := os.Open(fileName) + if err != nil { + return err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) + + if err != nil { + return err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil } -func (s *sources) CreateSourceWithURL(config *SourceConfig, pkgUrl string) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) - // buffer to store our request as bytes - bodyBuf := bytes.NewBufferString("") - - multiPartWriter := multipart.NewWriter(bodyBuf) - - textWriter, err := s.createTextFromFiled(multiPartWriter, "url") - if err != nil { - return err - } - - _, err = textWriter.Write([]byte(pkgUrl)) - if err != nil { - return err - } - - jsonData, err := json.Marshal(config) - if err != nil { - return err - } - - stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") - if err != nil { - return err - } - - _, err = stringWriter.Write(jsonData) - if err != nil { - return err - } - - if err = multiPartWriter.Close(); err != nil { - return err - } - - contentType := multiPartWriter.FormDataContentType() - err = s.client.postWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) - if err != nil { - return err - } - - return nil +func (s *sources) CreateSourceWithURL(config *SourceConfig, pkgURL string) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + textWriter, err := s.createTextFromFiled(multiPartWriter, "url") + if err != nil { + return err + } + + _, err = textWriter.Write([]byte(pkgURL)) + if err != nil { + return err + } + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil } func (s *sources) UpdateSource(config *SourceConfig, fileName string, updateOptions *UpdateOptions) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) - // buffer to store our request as bytes - bodyBuf := bytes.NewBufferString("") - - multiPartWriter := multipart.NewWriter(bodyBuf) - - jsonData, err := json.Marshal(config) - if err != nil { - return err - } - - stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") - if err != nil { - return err - } - - _, err = stringWriter.Write(jsonData) - if err != nil { - return err - } - - if updateOptions != nil { - updateData, err := json.Marshal(updateOptions) - if err != nil { - return err - } - - updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") - if err != nil { - return err - } - - _, err = updateStrWriter.Write(updateData) - if err != nil { - return err - } - } - - if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { - // If the function code is built in, we don't need to submit here - file, err := os.Open(fileName) - if err != nil { - return err - } - defer file.Close() - - part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) - - if err != nil { - return err - } - - // copy the actual file content to the filed's writer - _, err = io.Copy(part, file) - if err != nil { - return err - } - } - - // In here, we completed adding the file and the fields, let's close the multipart writer - // So it writes the ending boundary - if err = multiPartWriter.Close(); err != nil { - return err - } - - contentType := multiPartWriter.FormDataContentType() - err = s.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) - if err != nil { - return err - } - - return nil + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if updateOptions != nil { + updateData, err := json.Marshal(updateOptions) + if err != nil { + return err + } + + updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") + if err != nil { + return err + } + + _, err = updateStrWriter.Write(updateData) + if err != nil { + return err + } + } + + if fileName != "" && !strings.HasPrefix(fileName, "builtin://") { + // If the function code is built in, we don't need to submit here + file, err := os.Open(fileName) + if err != nil { + return err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("data", filepath.Base(file.Name())) + + if err != nil { + return err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.putWithMultiPart(endpoint, bodyBuf, contentType) + if err != nil { + return err + } + + return nil } -func (s *sources) UpdateSourceWithUrl(config *SourceConfig, pkgUrl string, updateOptions *UpdateOptions) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) - // buffer to store our request as bytes - bodyBuf := bytes.NewBufferString("") - - multiPartWriter := multipart.NewWriter(bodyBuf) - - textWriter, err := s.createTextFromFiled(multiPartWriter, "url") - if err != nil { - return err - } - - _, err = textWriter.Write([]byte(pkgUrl)) - if err != nil { - return err - } - - jsonData, err := json.Marshal(config) - if err != nil { - return err - } - - stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") - if err != nil { - return err - } - - _, err = stringWriter.Write(jsonData) - if err != nil { - return err - } - - if updateOptions != nil { - updateData, err := json.Marshal(updateOptions) - if err != nil { - return err - } - - updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") - if err != nil { - return err - } - - _, err = updateStrWriter.Write(updateData) - if err != nil { - return err - } - } - - // In here, we completed adding the file and the fields, let's close the multipart writer - // So it writes the ending boundary - if err = multiPartWriter.Close(); err != nil { - return err - } - - contentType := multiPartWriter.FormDataContentType() - err = s.client.putWithMultiPart(endpoint, nil, nil, bodyBuf, contentType) - if err != nil { - return err - } - - return nil +func (s *sources) UpdateSourceWithURL(config *SourceConfig, pkgURL string, updateOptions *UpdateOptions) error { + endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + textWriter, err := s.createTextFromFiled(multiPartWriter, "url") + if err != nil { + return err + } + + _, err = textWriter.Write([]byte(pkgURL)) + if err != nil { + return err + } + + jsonData, err := json.Marshal(config) + if err != nil { + return err + } + + stringWriter, err := s.createStringFromField(multiPartWriter, "sourceConfig") + if err != nil { + return err + } + + _, err = stringWriter.Write(jsonData) + if err != nil { + return err + } + + if updateOptions != nil { + updateData, err := json.Marshal(updateOptions) + if err != nil { + return err + } + + updateStrWriter, err := s.createStringFromField(multiPartWriter, "updateOptions") + if err != nil { + return err + } + + _, err = updateStrWriter.Write(updateData) + if err != nil { + return err + } + } + + // In here, we completed adding the file and the fields, let's close the multipart writer + // So it writes the ending boundary + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = s.client.putWithMultiPart(endpoint, bodyBuf, contentType) + if err != nil { + return err + } + + return nil } func (s *sources) DeleteSource(tenant, namespace, source string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - return s.client.delete(endpoint, nil) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + return s.client.delete(endpoint) } func (s *sources) GetSourceStatus(tenant, namespace, source string) (SourceStatus, error) { - var sourceStatus SourceStatus - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - err := s.client.get(endpoint+"/status", &sourceStatus) - return sourceStatus, err + var sourceStatus SourceStatus + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + err := s.client.get(endpoint+"/status", &sourceStatus) + return sourceStatus, err } func (s *sources) GetSourceStatusWithID(tenant, namespace, source string, id int) (SourceInstanceStatusData, error) { - var sourceInstanceStatusData SourceInstanceStatusData - instanceID := fmt.Sprintf("%d", id) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, instanceID) - err := s.client.get(endpoint+"/status", &sourceInstanceStatusData) - return sourceInstanceStatusData, err + var sourceInstanceStatusData SourceInstanceStatusData + instanceID := fmt.Sprintf("%d", id) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, instanceID) + err := s.client.get(endpoint+"/status", &sourceInstanceStatusData) + return sourceInstanceStatusData, err } func (s *sources) RestartSource(tenant, namespace, source string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - return s.client.post(endpoint+"/restart", "", nil) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + return s.client.post(endpoint+"/restart", "") } func (s *sources) RestartSourceWithID(tenant, namespace, source string, instanceID int) error { - id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) - return s.client.post(endpoint+"/restart", "", nil) + return s.client.post(endpoint+"/restart", "") } func (s *sources) StopSource(tenant, namespace, source string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - return s.client.post(endpoint+"/stop", "", nil) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + return s.client.post(endpoint+"/stop", "") } func (s *sources) StopSourceWithID(tenant, namespace, source string, instanceID int) error { - id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) - return s.client.post(endpoint+"/stop", "", nil) + return s.client.post(endpoint+"/stop", "") } func (s *sources) StartSource(tenant, namespace, source string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - return s.client.post(endpoint+"/start", "", nil) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) + return s.client.post(endpoint+"/start", "") } func (s *sources) StartSourceWithID(tenant, namespace, source string, instanceID int) error { - id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) + id := fmt.Sprintf("%d", instanceID) + endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) - return s.client.post(endpoint+"/start", "", nil) + return s.client.post(endpoint+"/start", "") } func (s *sources) GetBuiltInSources() ([]*ConnectorDefinition, error) { - var connectorDefinition []*ConnectorDefinition - endpoint := s.client.endpoint(s.basePath, "builtinsources") - err := s.client.get(endpoint, &connectorDefinition) - return connectorDefinition, err + var connectorDefinition []*ConnectorDefinition + endpoint := s.client.endpoint(s.basePath, "builtinsources") + err := s.client.get(endpoint, &connectorDefinition) + return connectorDefinition, err } func (s *sources) ReloadBuiltInSources() error { - endpoint := s.client.endpoint(s.basePath, "reloadBuiltInSources") - return s.client.post(endpoint, "", nil) + endpoint := s.client.endpoint(s.basePath, "reloadBuiltInSources") + return s.client.post(endpoint, "") } diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 0b1260a508..0533319bb0 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -34,7 +34,7 @@ type Topics interface { RevokePermission(TopicName, string) error Lookup(TopicName) (LookupData, error) GetBundleRange(TopicName) (string, error) - GetLastMessageId(TopicName) (MessageId, error) + GetLastMessageID(TopicName) (MessageID, error) GetStats(TopicName) (TopicStats, error) GetInternalStats(TopicName) (PersistentTopicInternalStats, error) GetPartitionedStats(TopicName, bool) (PartitionedTopicStats, error) @@ -63,7 +63,7 @@ func (t *topics) Create(topic TopicName, partitions int) error { if partitions == 0 { endpoint = t.client.endpoint(t.basePath, topic.GetRestPath()) } - return t.client.put(endpoint, partitions, nil) + return t.client.put(endpoint, partitions) } func (t *topics) Delete(topic TopicName, force bool, nonPartitioned bool) error { @@ -79,7 +79,7 @@ func (t *topics) Delete(topic TopicName, force bool, nonPartitioned bool) error func (t *topics) Update(topic TopicName, partitions int) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") - return t.client.post(endpoint, partitions, nil) + return t.client.post(endpoint, partitions) } func (t *topics) GetMetadata(topic TopicName) (PartitionedTopicMetadata, error) { @@ -137,7 +137,7 @@ func (t *topics) GetInternalInfo(topic TopicName) (ManagedLedgerInfo, error) { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "internal-info") var info ManagedLedgerInfo err := t.client.get(endpoint, &info) - return info, err + return info, err } func (t *topics) GetPermissions(topic TopicName) (map[string][]AuthAction, error) { @@ -149,16 +149,16 @@ func (t *topics) GetPermissions(topic TopicName) (map[string][]AuthAction, error func (t *topics) GrantPermission(topic TopicName, role string, action []AuthAction) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions", role) - var s []string + s := []string{} for _, v := range action { s = append(s, v.String()) } - return t.client.post(endpoint, s, nil) + return t.client.post(endpoint, s) } func (t *topics) RevokePermission(topic TopicName, role string) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions", role) - return t.client.delete(endpoint, nil) + return t.client.delete(endpoint) } func (t *topics) Lookup(topic TopicName) (LookupData, error) { @@ -174,11 +174,11 @@ func (t *topics) GetBundleRange(topic TopicName) (string, error) { return string(data), err } -func (t *topics) GetLastMessageId(topic TopicName) (MessageId, error) { - var messageId MessageId +func (t *topics) GetLastMessageID(topic TopicName) (MessageID, error) { + var messageID MessageID endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "lastMessageId") - err := t.client.get(endpoint, &messageId) - return messageId, err + err := t.client.get(endpoint, &messageID) + return messageID, err } func (t *topics) GetStats(topic TopicName) (TopicStats, error) { diff --git a/pulsaradmin/pkg/pulsar/topic_domain.go b/pulsaradmin/pkg/pulsar/topic_domain.go index a54d1a1abf..9210a507e5 100644 --- a/pulsaradmin/pkg/pulsar/topic_domain.go +++ b/pulsaradmin/pkg/pulsar/topic_domain.go @@ -22,8 +22,8 @@ import "github.com/pkg/errors" type TopicDomain string const ( - persistent TopicDomain = "persistent" - non_persistent TopicDomain = "non-persistent" + persistent TopicDomain = "persistent" + nonPersistent TopicDomain = "non-persistent" ) func ParseTopicDomain(domain string) (TopicDomain, error) { @@ -31,9 +31,9 @@ func ParseTopicDomain(domain string) (TopicDomain, error) { case "persistent": return persistent, nil case "non-persistent": - return non_persistent, nil + return nonPersistent, nil default: - return "", errors.Errorf("The domain only can be specified as 'persistent' or " + + return "", errors.Errorf("The domain only can be specified as 'persistent' or "+ "'non-persistent'. Input domain is '%s'.", domain) } } diff --git a/pulsaradmin/pkg/pulsar/topic_name.go b/pulsaradmin/pkg/pulsar/topic_name.go index 5c27eb1501..324e9d82fd 100644 --- a/pulsaradmin/pkg/pulsar/topic_name.go +++ b/pulsaradmin/pkg/pulsar/topic_name.go @@ -19,16 +19,17 @@ package pulsar import ( "fmt" - "github.com/pkg/errors" "net/url" "strconv" "strings" + + "github.com/pkg/errors" ) const ( - PUBLIC_TENANT = "public" - DEFAULT_NAMESPACE = "default" - PARTITIONED_TOPIC_SUFFIX = "-partition-" + PUBLICTENANT = "public" + DEFAULTNAMESPACE = "default" + PARTITIONEDTOPICSUFFIX = "-partition-" ) type TopicName struct { @@ -50,11 +51,12 @@ func GetTopicName(completeName string) (*TopicName, error) { // - // if !strings.Contains(completeName, "://") { parts := strings.Split(completeName, "/") - if len(parts) == 3 { + switch len(parts) { + case 3: completeName = persistent.String() + "://" + completeName - } else if len(parts) == 1 { - completeName = persistent.String() + "://" + PUBLIC_TENANT + "/" + DEFAULT_NAMESPACE + "/" + parts[0] - } else { + case 1: + completeName = persistent.String() + "://" + PUBLICTENANT + "/" + DEFAULTNAMESPACE + "/" + parts[0] + default: return nil, errors.Errorf("Invalid short topic name '%s', it should be "+ "in the format of // or ", completeName) } @@ -79,12 +81,12 @@ func GetTopicName(completeName string) (*TopicName, error) { topicname.topic = parts[2] topicname.partitionIndex = getPartitionIndex(completeName) } else { - return nil, errors.Errorf("Invalid topic name '%s', it should be in the format of "+ + return nil, errors.Errorf("invalid topic name '%s', it should be in the format of "+ "//", rest) } if topicname.topic == "" { - return nil, errors.New("Topic name can not be empty.") + return nil, errors.New("topic name can not be empty") } n, err := GetNameSpaceName(topicname.tenant, topicname.namespace) @@ -118,19 +120,19 @@ func (t *TopicName) GetLocalName() string { func (t *TopicName) GetPartition(index int) (*TopicName, error) { if index < 0 { - return nil, errors.New("Invalid partition index number.") + return nil, errors.New("invalid partition index number") } - if strings.Contains(t.String(), PARTITIONED_TOPIC_SUFFIX) { + if strings.Contains(t.String(), PARTITIONEDTOPICSUFFIX) { return t, nil } - topicNameWithPartition := t.String() + PARTITIONED_TOPIC_SUFFIX + strconv.Itoa(index) + topicNameWithPartition := t.String() + PARTITIONEDTOPICSUFFIX + strconv.Itoa(index) return GetTopicName(topicNameWithPartition) } func getPartitionIndex(topic string) int { - if strings.Contains(topic, PARTITIONED_TOPIC_SUFFIX) { + if strings.Contains(topic, PARTITIONEDTOPICSUFFIX) { parts := strings.Split(topic, "-") index, err := strconv.Atoi(parts[len(parts)-1]) if err == nil { diff --git a/pulsaradmin/pkg/pulsar/topic_name_test.go b/pulsaradmin/pkg/pulsar/topic_name_test.go index a7be0d03f3..55fc5090cd 100644 --- a/pulsaradmin/pkg/pulsar/topic_name_test.go +++ b/pulsaradmin/pkg/pulsar/topic_name_test.go @@ -18,9 +18,10 @@ package pulsar import ( - "github.com/stretchr/testify/assert" "net/url" "testing" + + "github.com/stretchr/testify/assert" ) func TestGetTopicName(t *testing.T) { @@ -40,11 +41,12 @@ func TestGetTopicName(t *testing.T) { assert.Nil(t, err) assert.Equal(t, "non-persistent://tenant/namespace/success", success.String()) - fail, err := GetTopicName("://tenant.namespace.topic") + _, err = GetTopicName("://tenant.namespace.topic") assert.NotNil(t, err) - assert.Equal(t, "The domain only can be specified as 'persistent' or 'non-persistent'. Input domain is ''.", err.Error()) + assert.Equal(t, "The domain only can be specified as 'persistent' or 'non-persistent'."+ + " Input domain is ''.", err.Error()) - fail, err = GetTopicName("default/fail") + fail, err := GetTopicName("default/fail") assert.NotNil(t, err) assert.Equal(t, "Invalid short topic name 'default/fail', it should be in the "+ "format of // or ", err.Error()) @@ -63,13 +65,13 @@ func TestGetTopicName(t *testing.T) { fail, err = GetTopicName("persistent://tenant/namespace") assert.NotNil(t, err) - assert.Equal(t, "Invalid topic name 'tenant/namespace', it should be in the format "+ + assert.Equal(t, "invalid topic name 'tenant/namespace', it should be in the format "+ "of //", err.Error()) assert.Nil(t, fail) fail, err = GetTopicName("persistent://tenant/namespace/") assert.NotNil(t, err) - assert.Equal(t, "Topic name can not be empty.", err.Error()) + assert.Equal(t, "topic name can not be empty", err.Error()) assert.Nil(t, fail) } diff --git a/pulsaradmin/pkg/pulsar/utils.go b/pulsaradmin/pkg/pulsar/utils.go index e408745167..249f18d23f 100644 --- a/pulsaradmin/pkg/pulsar/utils.go +++ b/pulsaradmin/pkg/pulsar/utils.go @@ -21,6 +21,6 @@ import ( "fmt" ) -func makeHttpPath(apiVersion string, componentPath string) string { +func makeHTTPPath(apiVersion string, componentPath string) string { return fmt.Sprintf("/admin/%s%s", apiVersion, componentPath) } From 8a091d2138909901b664d49ba0b159e29b93ea72 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Tue, 8 Oct 2019 12:01:38 +0800 Subject: [PATCH 124/348] Add website for pulsarctl (streamnative/pulsarctl#77) * Add website for pulsarctl Signed-off-by: xiaolong.ran --- pulsaradmin/pkg/pulsar/descriptions.go | 8 ++++---- pulsaradmin/pkg/pulsar/descriptions_test.go | 5 +---- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/descriptions.go b/pulsaradmin/pkg/pulsar/descriptions.go index 9a5a3e16f1..44c933bafe 100644 --- a/pulsaradmin/pkg/pulsar/descriptions.go +++ b/pulsaradmin/pkg/pulsar/descriptions.go @@ -17,7 +17,9 @@ package pulsar -import "strings" +import ( + "strings" +) var SPACES = " " var USEDFOR = "USED FOR:" @@ -47,13 +49,11 @@ func (desc *LongDescription) ToString() string { SPACES + desc.CommandUsedFor + "\n\n" + PERMISSION + "\n" + SPACES + desc.CommandPermission + "\n\n" + - EXAMPLES + "\n" + - desc.exampleToString() + OUTPUT + "\n" + desc.outputToString() } -func (desc *LongDescription) exampleToString() string { +func (desc *LongDescription) ExampleToString() string { var result string for _, v := range desc.CommandExamples { result += SPACES + "#" + v.Desc + "\n" + SPACES + v.Command + "\n\n" diff --git a/pulsaradmin/pkg/pulsar/descriptions_test.go b/pulsaradmin/pkg/pulsar/descriptions_test.go index 23ba0d82c9..361fdd5f2c 100644 --- a/pulsaradmin/pkg/pulsar/descriptions_test.go +++ b/pulsaradmin/pkg/pulsar/descriptions_test.go @@ -30,7 +30,7 @@ func TestLongDescription_exampleToString(t *testing.T) { Command: "command", } desc.CommandExamples = []Example{example} - res := desc.exampleToString() + res := desc.ExampleToString() expect := " #command description\n" + " command\n\n" @@ -56,9 +56,6 @@ func TestLongDescription_ToString(t *testing.T) { " " + desc.CommandUsedFor + "\n\n" + "REQUIRED PERMISSION:\n" + " " + desc.CommandPermission + "\n\n" + - "EXAMPLES:\n" + - " " + "#" + example.Desc + "\n" + - " " + example.Command + "\n\n" + "OUTPUT:\n" + " " + "#" + out.Desc + "\n" + " " + "Out line 1" + "\n" + From 608b23dcd80fc2f457fd5959635eeacbee1fad2b Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 10 Oct 2019 03:56:56 +0800 Subject: [PATCH 125/348] Add topic command `compact` and `compact-status` (streamnative/pulsarctl#75) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit *Motivation* - Add command `compact` - Add command `compact-status` *compact* ``` USED FOR: This command is used for compacting a persistent topic. REQUIRED PERMISSION: This command is requires tenant admin permissions. EXAMPLES: #Compact a persistent topic pulsarctl topic compact OUTPUT: #normal output Sending compact topic request successfully #the topic name is not specified [✖] only one argument is allowed to be used as a name #the specified topic does not found [✖] code: 404 reason: Topic not found #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics compact [flags] ``` *compact-status* ``` USED FOR: This command is used for getting status of compaction on a topic. REQUIRED PERMISSION: This command requires tenant admin permissions. EXAMPLES: #Get status of compaction of a persistent topic pulsarctl topic compact-status OUTPUT: #normal output Compacting the topic is done successfully #Compacting the topic is not running Compacting the topic is not running #Compacting the topic is running Compacting the topic is running #Compacting the topic is done with error Compacting the topic is done with error #the topic name is not specified [✖] only one argument is allowed to be used as a name #the specified topic does not found [✖] code: 404 reason: Topic not found #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics compact-status [flags] ``` --- .../pkg/pulsar/long_running_process_status.go | 42 +++++++++++++++++++ pulsaradmin/pkg/pulsar/topic.go | 14 +++++++ pulsaradmin/pkg/pulsar/topic_name.go | 4 ++ 3 files changed, 60 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/long_running_process_status.go diff --git a/pulsaradmin/pkg/pulsar/long_running_process_status.go b/pulsaradmin/pkg/pulsar/long_running_process_status.go new file mode 100644 index 0000000000..177f14bd5b --- /dev/null +++ b/pulsaradmin/pkg/pulsar/long_running_process_status.go @@ -0,0 +1,42 @@ +// 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 pulsar + +type Status string + +const ( + NOTRUN Status = "NOT_RUN" + RUNNING Status = "RUNNING" + SUCCESS Status = "SUCCESS" + ERROR Status = "ERROR" +) + +type LongRunningProcessStatus struct { + Status Status `json:"status"` + LastError string `json:"lastError"` +} + +type OffloadProcessStatus struct { + Status Status `json:"status"` + LastError string `json:"lastError"` + FirstUnOffloadedMessage MessageID `json:"firstUnoffloadedMessage"` +} + +func (s Status) String() string { + return string(s) +} diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 0533319bb0..faac080a4e 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -38,6 +38,8 @@ type Topics interface { GetStats(TopicName) (TopicStats, error) GetInternalStats(TopicName) (PersistentTopicInternalStats, error) GetPartitionedStats(TopicName, bool) (PartitionedTopicStats, error) + Compact(TopicName) error + CompactStatus(TopicName) (LongRunningProcessStatus, error) } type topics struct { @@ -204,3 +206,15 @@ func (t *topics) GetPartitionedStats(topic TopicName, perPartition bool) (Partit _, err := t.client.getWithQueryParams(endpoint, &stats, params, true) return stats, err } + +func (t *topics) Compact(topic TopicName) error { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "compaction") + return t.client.put(endpoint, "") +} + +func (t *topics) CompactStatus(topic TopicName) (LongRunningProcessStatus, error) { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "compaction") + var status LongRunningProcessStatus + err := t.client.get(endpoint, &status) + return status, err +} diff --git a/pulsaradmin/pkg/pulsar/topic_name.go b/pulsaradmin/pkg/pulsar/topic_name.go index 324e9d82fd..f18db1d060 100644 --- a/pulsaradmin/pkg/pulsar/topic_name.go +++ b/pulsaradmin/pkg/pulsar/topic_name.go @@ -106,6 +106,10 @@ func (t *TopicName) GetDomain() TopicDomain { return t.domain } +func (t *TopicName) IsPersistent() bool { + return t.domain == persistent +} + func (t *TopicName) GetRestPath() string { return fmt.Sprintf("%s/%s/%s/%s", t.domain, t.tenant, t.namespace, t.GetEncodedTopic()) } From f27c2ec67beb59db4d268e096f095bf0744b76e9 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 10 Oct 2019 12:51:18 +0800 Subject: [PATCH 126/348] Add topic command `unload` (streamnative/pulsarctl#76) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Add topic command `unload` USED FOR: This command is used for unloading a topic. REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #Unload a topic pulsarctl topic unload OUTPUT: #normal output Unload topic successfully #the topic name is not specified [✖] only one argument is allowed to be used as a name #the specified topic does not found [✖] code: 404 reason: Topic not found #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics unload [flags] --- pulsaradmin/pkg/pulsar/topic.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index faac080a4e..6b9c10e925 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -38,6 +38,7 @@ type Topics interface { GetStats(TopicName) (TopicStats, error) GetInternalStats(TopicName) (PersistentTopicInternalStats, error) GetPartitionedStats(TopicName, bool) (PartitionedTopicStats, error) + Unload(TopicName) error Compact(TopicName) error CompactStatus(TopicName) (LongRunningProcessStatus, error) } @@ -207,6 +208,10 @@ func (t *topics) GetPartitionedStats(topic TopicName, perPartition bool) (Partit return stats, err } +func (t *topics) Unload(topic TopicName) error { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "unload") + return t.client.put(endpoint, "") +} func (t *topics) Compact(topic TopicName) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "compaction") return t.client.put(endpoint, "") From 9ac284471489293b3ee93b9b2511eda7761ac42b Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 10 Oct 2019 18:41:48 +0800 Subject: [PATCH 127/348] Add namespaces permissions commands (streamnative/pulsarctl#71) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- *Motivation* - Add commands `permissions` - Add commands `grant` - Add commands `revoke` - Add commands `grant-sub` - Add commands `revoke-sub` --- *OUTPUT* - permissions ``` USED FOR: This command is used for getting permissions configure data of a namespace. REQUIRED PERMISSION: This command requires tenant admin permissions. EXAMPLES: #Get permissions configure data of a namespace / pulsarctl namespaces permissions / OUTPUT: #normal output { "": [ "" ] } #the namespace name is not specified [✖] only one argument is allowed to be used as a name #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl namespaces permissions [flags] ``` - grant ``` USED FOR: This command is used for granting permissions to a client role on a namespace. REQUIRED PERMISSION: This command requires tenant admin permissions. EXAMPLES: #Grant permission to the client role on the namespace pulsarctl namespaces grant --role --actions #Grant permissions to the client role on the namespace pulsarctl namespaces grant --role --actions --actions OUTPUT: #normal output Grant permissions to the client role on the namespace successfully #the namespace name is not specified [✖] only one argument is allowed to be used as a name #the authorization is not enabled [✖] code: 501 reason: Authorization is not enabled #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl namespaces grant [flags] Grant Permissions flags: --role string Client role to which grant permissions --actions strings Actions to be granted (produce,consume,functions) ``` - revoke ``` USED FOR: This command is used for revoking a client role permissions on a namespace. REQUIRED PERMISSION: This command requires tenant admin permissions and broker has read-writer permissions on the zookeeper. EXAMPLES: #Revoke the client role on the namespace pulsarctl namespaces revoke --role OUTPUT: #normal output Revoke the client role permissions on the namespace successfully #the namespace name is not specified [✖] only one argument is allowed to be used as a name #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl namespaces revoke [flags] Revoke Permissions flags: --role string Client role to which revoke permissions ``` - grant-sub ``` USED FOR: This command is used for granting client roles to a subscription of a namespace. REQUIRED PERMISSION: This command requires super-user permissions. EXAMPLES: #Grant client roles to the subscription of the namespace pulsarctl namespaces grant-sub --role --role OUTPUT: #normal output Grant client roles to the subscription of the namespace successfully #the namespace name is not specified or the subscription name is not specified [✖] need to specified namespace name and subscription name #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl namespaces grant-sub [flags] Grant Subscription Permissions flags: --role strings Client role to which grant permissions ``` - revoke-sub ``` USED FOR: This command is used for revoking a client role permissions on a subscription of a namespace. REQUIRED PERMISSION: This command requires tenant admin permissions and broker has read-writer permissions on the zookeeper. EXAMPLES: #Revoke a client role on the subscription of the pulsarctl namespaces revoke --role OUTPUT: #normal output Revoke the client role permissions on the subscription of the namespace successfully #the namespace name is not specified or the subscription name is not specified [✖] need to specified namespace name and subscription name #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl namespaces revoke-sub [flags] Revoke Subscription Permissions flags: --role string Client role to which revoke permissions ``` --- pulsaradmin/pkg/pulsar/auth_action.go | 2 +- pulsaradmin/pkg/pulsar/namespace.go | 39 +++++++++++++++++++++++++++ 2 files changed, 40 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/auth_action.go b/pulsaradmin/pkg/pulsar/auth_action.go index 1bf9fe1e68..84e42debbb 100644 --- a/pulsaradmin/pkg/pulsar/auth_action.go +++ b/pulsaradmin/pkg/pulsar/auth_action.go @@ -36,7 +36,7 @@ func ParseAuthAction(action string) (AuthAction, error) { case "functions": return functionsAuth, nil default: - return "", errors.Errorf("The auth action only can be specified as 'produce', "+ + return "", errors.Errorf("The auth action only can be specified as 'produce', "+ "'consume', or 'functions'. Invalid auth action '%s'", action) } } diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index d1e29a2f58..20e88a48fa 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -118,6 +118,12 @@ type Namespaces interface { // Split namespace bundle SplitNamespaceBundle(namespace, bundle string, unloadSplitBundles bool) error + GetNamespacePermissions(namespace NameSpaceName) (map[string][]AuthAction, error) + GrantNamespacePermission(namespace NameSpaceName, role string, action []AuthAction) error + RevokeNamespacePermission(namespace NameSpaceName, role string) error + GrantSubPermission(namespace NameSpaceName, sName string, roles []string) error + RevokeSubPermission(namespace NameSpaceName, sName, role string) error + // Set the given subscription auth mode on all topics on a namespace SetSubscriptionAuthMode(namespace NameSpaceName, mode SubscriptionAuthMode) error @@ -483,6 +489,39 @@ func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitB return n.client.putWithQueryParams(endpoint, "", nil, params) } +func (n *namespaces) GetNamespacePermissions(namespace NameSpaceName) (map[string][]AuthAction, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions") + var permissions map[string][]AuthAction + err := n.client.get(endpoint, &permissions) + return permissions, err +} + +func (n *namespaces) GrantNamespacePermission(namespace NameSpaceName, role string, action []AuthAction) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", role) + s := make([]string, 0) + for _, v := range action { + s = append(s, v.String()) + } + return n.client.post(endpoint, s) +} + +func (n *namespaces) RevokeNamespacePermission(namespace NameSpaceName, role string) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", role) + return n.client.delete(endpoint) +} + +func (n *namespaces) GrantSubPermission(namespace NameSpaceName, sName string, roles []string) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", + "subscription", sName) + return n.client.post(endpoint, roles) +} + +func (n *namespaces) RevokeSubPermission(namespace NameSpaceName, sName, role string) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", + "subscription", sName, role) + return n.client.delete(endpoint) +} + func (n *namespaces) SetSubscriptionAuthMode(namespace NameSpaceName, mode SubscriptionAuthMode) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionAuthMode") return n.client.post(endpoint, mode.String()) From 963abf5f186cea1e754d568ce7126683534e7226 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 10 Oct 2019 18:50:20 +0800 Subject: [PATCH 128/348] Add topic command `offload` and `offload-status` (streamnative/pulsarctl#74) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit *Motivation* - Add command `offload` - Add command `offload-status` *offload* ``` USED FOR: This command is used for triggering offloading the data from a topic to long-term storage (e.g. Amazon S3) REQUIRED PERMISSION: This command requires tenant admin permissions. EXAMPLES: #Trigger offloading the data from a topic to a long-term storage and keep the configured amount of data in BookKeeper only (e.g. 10M, 5G, default is byte) pulsarctl topic offload OUTPUT: #normal output Offload trigger for for messages before #noting to offload Nothing to offload #the topic name is not specified or the offload threshold is not specified [✖] only two argument is allowed to be used as names #the specified topic does not found [✖] code: 404 reason: Topic not found #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics offload [flags] ``` *offload-status* ``` USED FOR: This command is used for checking the status of data offloading from a persistent topic to long-term storage. REQUIRED PERMISSION: This command requires tenant admin permissions. EXAMPLES: #Check the status of data offloading from a topic to long-term storage pulsarctl topic offload-status #Wait for offloading to complete pulsarctl topic offload-status --wait OUTPUT: #normal output Offloading topic data is done successfully #Offloading topic is not running Offloading topic data is not running #Offloading topic is running Offloading topic data is running #Offloading topic with error Offloading topic data is done with error #the topic name is not specified [✖] only one argument is allowed to be used as a name #the specified topic does not found [✖] code: 404 reason: Topic not found #the topic name is not in the format of // or [✖] Invalid short topic name '', it should be in the format of // or #the topic name is not in the format of ://// [✖] Invalid complete topic name '', it should be in the format of ://// #the topic name is not in the format of // [✖] Invalid topic name '', it should be in the format of// #the namespace name is not in the format of / [✖] The complete name of namespace is invalid. complete name : #the tenant name and(or) namespace name is empty [✖] Invalid tenant or namespace. [/] #the tenant name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Tenant name include unsupported special chars. tenant : [] #the namespace name contains unsupported special chars. the alphanumeric (a-zA-Z0-9) and the special chars (-=:.%) is allowed [✖] Namespace name include unsupported special chars. namespace : [] Usage: pulsarctl topics offload-status [flags] OffloadStatus flags: -w, --wait Wait for offloading to complete ``` --- pulsaradmin/pkg/pulsar/topic.go | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 6b9c10e925..05ee0be15f 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -38,6 +38,8 @@ type Topics interface { GetStats(TopicName) (TopicStats, error) GetInternalStats(TopicName) (PersistentTopicInternalStats, error) GetPartitionedStats(TopicName, bool) (PartitionedTopicStats, error) + Offload(TopicName, MessageID) error + OffloadStatus(TopicName) (OffloadProcessStatus, error) Unload(TopicName) error Compact(TopicName) error CompactStatus(TopicName) (LongRunningProcessStatus, error) @@ -208,10 +210,23 @@ func (t *topics) GetPartitionedStats(topic TopicName, perPartition bool) (Partit return stats, err } +func (t *topics) Offload(topic TopicName, messageID MessageID) error { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "offload") + return t.client.put(endpoint, messageID) +} + +func (t *topics) OffloadStatus(topic TopicName) (OffloadProcessStatus, error) { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "offload") + var status OffloadProcessStatus + err := t.client.get(endpoint, &status) + return status, err +} + func (t *topics) Unload(topic TopicName) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "unload") return t.client.put(endpoint, "") } + func (t *topics) Compact(topic TopicName) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "compaction") return t.client.put(endpoint, "") From a4da6e5d5a59a009e6456de8a3150e7de98d7c92 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 11 Oct 2019 14:22:41 +0800 Subject: [PATCH 129/348] Add topic command `terminate` (streamnative/pulsarctl#73) * Add topic command `terminate` --- pulsaradmin/pkg/pulsar/admin.go | 9 +++++++++ pulsaradmin/pkg/pulsar/topic.go | 8 ++++++++ 2 files changed, 17 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 4f560e411d..e1ad829ad7 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -274,6 +274,10 @@ func (c *client) deleteWithQueryParams(endpoint string, obj interface{}, params } func (c *client) post(endpoint string, in interface{}) error { + return c.postWithObj(endpoint, in, nil) +} + +func (c *client) postWithObj(endpoint string, in, obj interface{}) error { req, err := c.newRequest(http.MethodPost, endpoint) if err != nil { return err @@ -286,6 +290,11 @@ func (c *client) post(endpoint string, in interface{}) error { return err } defer safeRespClose(resp) + if obj != nil { + if err := decodeJSONBody(resp, &obj); err != nil { + return err + } + } return nil } diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 05ee0be15f..ec00a43532 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -38,6 +38,7 @@ type Topics interface { GetStats(TopicName) (TopicStats, error) GetInternalStats(TopicName) (PersistentTopicInternalStats, error) GetPartitionedStats(TopicName, bool) (PartitionedTopicStats, error) + Terminate(TopicName) (MessageID, error) Offload(TopicName, MessageID) error OffloadStatus(TopicName) (OffloadProcessStatus, error) Unload(TopicName) error @@ -210,6 +211,13 @@ func (t *topics) GetPartitionedStats(topic TopicName, perPartition bool) (Partit return stats, err } +func (t *topics) Terminate(topic TopicName) (MessageID, error) { + endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "terminate") + var messageID MessageID + err := t.client.postWithObj(endpoint, "", &messageID) + return messageID, err +} + func (t *topics) Offload(topic TopicName, messageID MessageID) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "offload") return t.client.put(endpoint, messageID) From 4e8331d071c82ca86f445e51417dfaeafa01ee41 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Sat, 12 Oct 2019 16:42:43 +0800 Subject: [PATCH 130/348] Add set and get commands for MaxConsumersPerSub and MaxConsumersPerTopic and maxProduersPerTopic (streamnative/pulsarctl#67) *Modifications* Add command `set-max-consumers-per-subscription` Add command `get-max-consumers-per-subscription` Add command `set-max-consumers-per-topic` Add command `get-max-consumers-per-topic` Add command `set-max-producers-per-topic` Add command `get-max-producers-per-topic` --- pulsaradmin/pkg/pulsar/namespace.go | 60 +++++++++++++++++++++++++++++ 1 file changed, 60 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index 20e88a48fa..92ab780820 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -71,6 +71,24 @@ type Namespaces interface { // Remove a backlog quota policy from a namespace RemoveBacklogQuota(namespace string) error + // Set maxConsumersPerSubscription for a namespace. + SetMaxConsumersPerSubscription(namespace NameSpaceName, max int) error + + // Get the maxConsumersPerSubscription for a namespace. + GetMaxConsumersPerSubscription(namespace NameSpaceName) (int, error) + + // Set maxConsumersPerTopic for a namespace. + SetMaxConsumersPerTopic(namespace NameSpaceName, max int) error + + // Get the maxProducersPerTopic for a namespace. + GetMaxConsumersPerTopic(namespace NameSpaceName) (int, error) + + // Set maxProducersPerTopic for a namespace. + SetMaxProducersPerTopic(namespace NameSpaceName, max int) error + + // Get the maxProducersPerTopic for a namespace. + GetMaxProducersPerTopic(namespace NameSpaceName) (int, error) + // Get the replication clusters for a namespace GetNamespaceReplicationClusters(namespace string) ([]string, error) @@ -343,6 +361,48 @@ func (n *namespaces) RemoveBacklogQuota(namespace string) error { return n.client.deleteWithQueryParams(endpoint, nil, params) } +func (n *namespaces) SetMaxConsumersPerSubscription(namespace NameSpaceName, max int) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerSubscription") + return n.client.post(endpoint, max) +} + +func (n *namespaces) GetMaxConsumersPerSubscription(namespace NameSpaceName) (int, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerSubscription") + b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + if err != nil { + return -1, err + } + return strconv.Atoi(string(b)) +} + +func (n *namespaces) SetMaxConsumersPerTopic(namespace NameSpaceName, max int) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerTopic") + return n.client.post(endpoint, max) +} + +func (n *namespaces) GetMaxConsumersPerTopic(namespace NameSpaceName) (int, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerTopic") + b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + if err != nil { + return -1, err + } + return strconv.Atoi(string(b)) +} + +func (n *namespaces) SetMaxProducersPerTopic(namespace NameSpaceName, max int) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxProducersPerTopic") + return n.client.post(endpoint, max) +} + +func (n *namespaces) GetMaxProducersPerTopic(namespace NameSpaceName) (int, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxProducersPerTopic") + b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + if err != nil { + return -1, err + } + return strconv.Atoi(string(b)) +} + func (n *namespaces) GetNamespaceReplicationClusters(namespace string) ([]string, error) { var data []string nsName, err := GetNamespaceName(namespace) From 6aa9eb42cb8db1179b7bd470835e331174989795 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Sat, 12 Oct 2019 19:46:06 +0800 Subject: [PATCH 131/348] Add schema, offload and compaction commands of namespaces for pulsasrctl (streamnative/pulsarctl#64) Command List set-schema-validation-enforced get-schema-validation-enforced set-schema-autoupdate-strategy get-schema-autoupdate-strategy set-offload-threshold get-offload-threshold set-offload-deletion-lag get-offload-deletion-lag clear-offload-deletion-lag set-compaction-threshold get-compaction-threshold --- pulsaradmin/pkg/pulsar/namespace.go | 118 ++++++++++++++++++++++ pulsaradmin/pkg/pulsar/schema_strategy.go | 29 ++++++ 2 files changed, 147 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index 92ab780820..a1f96efcfe 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -20,6 +20,7 @@ package pulsar import ( "net/url" "strconv" + "strings" ) type Namespaces interface { @@ -71,6 +72,41 @@ type Namespaces interface { // Remove a backlog quota policy from a namespace RemoveBacklogQuota(namespace string) error + // Set schema validation enforced for namespace + SetSchemaValidationEnforced(namespace NameSpaceName, schemaValidationEnforced bool) error + + // Get schema validation enforced for namespace + GetSchemaValidationEnforced(namespace NameSpaceName) (bool, error) + + // Set the strategy used to check the a new schema provided by a producer is compatible with the current schema + // before it is installed + SetSchemaAutoUpdateCompatibilityStrategy(namespace NameSpaceName, strategy SchemaCompatibilityStrategy) error + + // Get the strategy used to check the a new schema provided by a producer is compatible with the current schema + // before it is installed + GetSchemaAutoUpdateCompatibilityStrategy(namespace NameSpaceName) (SchemaCompatibilityStrategy, error) + + // Clear the offload deletion lag for a namespace. + ClearOffloadDeleteLag(namespace NameSpaceName) error + + // Set the offload deletion lag for a namespace + SetOffloadDeleteLag(namespace NameSpaceName, timeMs int64) error + + // Get the offload deletion lag for a namespace, in milliseconds + GetOffloadDeleteLag(namespace NameSpaceName) (int64, error) + + // Set the offloadThreshold for a namespace + SetOffloadThreshold(namespace NameSpaceName, threshold int64) error + + // Get the offloadThreshold for a namespace + GetOffloadThreshold(namespace NameSpaceName) (int64, error) + + // Set the compactionThreshold for a namespace + SetCompactionThreshold(namespace NameSpaceName, threshold int64) error + + // Get the compactionThreshold for a namespace + GetCompactionThreshold(namespace NameSpaceName) (int64, error) + // Set maxConsumersPerSubscription for a namespace. SetMaxConsumersPerSubscription(namespace NameSpaceName, max int) error @@ -361,6 +397,60 @@ func (n *namespaces) RemoveBacklogQuota(namespace string) error { return n.client.deleteWithQueryParams(endpoint, nil, params) } +func (n *namespaces) SetSchemaValidationEnforced(namespace NameSpaceName, schemaValidationEnforced bool) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaValidationEnforced") + return n.client.post(endpoint, schemaValidationEnforced) +} + +func (n *namespaces) GetSchemaValidationEnforced(namespace NameSpaceName) (bool, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaValidationEnforced") + r, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + if err != nil { + return false, err + } + return strconv.ParseBool(string(r)) +} + +func (n *namespaces) SetSchemaAutoUpdateCompatibilityStrategy(namespace NameSpaceName, + strategy SchemaCompatibilityStrategy) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaAutoUpdateCompatibilityStrategy") + return n.client.put(endpoint, strategy.String()) +} + +func (n *namespaces) GetSchemaAutoUpdateCompatibilityStrategy(namespace NameSpaceName) (SchemaCompatibilityStrategy, + error) { + + endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaAutoUpdateCompatibilityStrategy") + b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + if err != nil { + return "", err + } + s, err := ParseSchemaAutoUpdateCompatibilityStrategy(strings.ReplaceAll(string(b), "\"", "")) + if err != nil { + return "", err + } + return s, nil +} + +func (n *namespaces) ClearOffloadDeleteLag(namespace NameSpaceName) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") + return n.client.delete(endpoint) +} + +func (n *namespaces) SetOffloadDeleteLag(namespace NameSpaceName, timeMs int64) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") + return n.client.put(endpoint, timeMs) +} + +func (n *namespaces) GetOffloadDeleteLag(namespace NameSpaceName) (int64, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") + b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + if err != nil { + return -1, err + } + return strconv.ParseInt(string(b), 10, 64) +} + func (n *namespaces) SetMaxConsumersPerSubscription(namespace NameSpaceName, max int) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerSubscription") return n.client.post(endpoint, max) @@ -375,6 +465,20 @@ func (n *namespaces) GetMaxConsumersPerSubscription(namespace NameSpaceName) (in return strconv.Atoi(string(b)) } +func (n *namespaces) SetOffloadThreshold(namespace NameSpaceName, threshold int64) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadThreshold") + return n.client.put(endpoint, threshold) +} + +func (n *namespaces) GetOffloadThreshold(namespace NameSpaceName) (int64, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadThreshold") + b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + if err != nil { + return -1, err + } + return strconv.ParseInt(string(b), 10, 64) +} + func (n *namespaces) SetMaxConsumersPerTopic(namespace NameSpaceName, max int) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerTopic") return n.client.post(endpoint, max) @@ -389,6 +493,20 @@ func (n *namespaces) GetMaxConsumersPerTopic(namespace NameSpaceName) (int, erro return strconv.Atoi(string(b)) } +func (n *namespaces) SetCompactionThreshold(namespace NameSpaceName, threshold int64) error { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "compactionThreshold") + return n.client.put(endpoint, threshold) +} + +func (n *namespaces) GetCompactionThreshold(namespace NameSpaceName) (int64, error) { + endpoint := n.client.endpoint(n.basePath, namespace.String(), "compactionThreshold") + b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + if err != nil { + return -1, err + } + return strconv.ParseInt(string(b), 10, 64) +} + func (n *namespaces) SetMaxProducersPerTopic(namespace NameSpaceName, max int) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxProducersPerTopic") return n.client.post(endpoint, max) diff --git a/pulsaradmin/pkg/pulsar/schema_strategy.go b/pulsaradmin/pkg/pulsar/schema_strategy.go index e0f4f2cdbe..309fd07334 100644 --- a/pulsaradmin/pkg/pulsar/schema_strategy.go +++ b/pulsaradmin/pkg/pulsar/schema_strategy.go @@ -17,6 +17,8 @@ package pulsar +import "github.com/pkg/errors" + type SchemaCompatibilityStrategy string const ( @@ -29,3 +31,30 @@ const ( ForwardTransitive SchemaCompatibilityStrategy = "ForwardTransitive" FullTransitive SchemaCompatibilityStrategy = "FullTransitive" ) + +func ParseSchemaAutoUpdateCompatibilityStrategy(str string) (SchemaCompatibilityStrategy, error) { + switch str { + case "AutoUpdateDisabled": + return AutoUpdateDisabled, nil + case "Backward": + return Backward, nil + case "Forward": + return Forward, nil + case "Full": + return Full, nil + case "AlwaysCompatible": + return AlwaysCompatible, nil + case "BackwardTransitive": + return BackwardTransitive, nil + case "ForwardTransitive": + return ForwardTransitive, nil + case "FullTransitive": + return FullTransitive, nil + default: + return "", errors.Errorf("Invalid auth strategy %s", str) + } +} + +func (s SchemaCompatibilityStrategy) String() string { + return string(s) +} From f5ae61c84b6d4855208be2e1ace36c5f10451c99 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 14 Oct 2019 14:55:46 +0800 Subject: [PATCH 132/348] Add subscription commands (streamnative/pulsarctl#58) Master issue: streamnative/pulsarctl#2 --- pulsaradmin/go.mod | 1 + pulsaradmin/pkg/pulsar/admin.go | 1 + pulsaradmin/pkg/pulsar/message.go | 88 ++++++++ pulsaradmin/pkg/pulsar/message_id.go | 58 +++++ pulsaradmin/pkg/pulsar/message_id_test.go | 65 ++++++ pulsaradmin/pkg/pulsar/namespace.go | 14 +- pulsaradmin/pkg/pulsar/sink_status.go | 2 +- pulsaradmin/pkg/pulsar/subscription.go | 246 ++++++++++++++++++++++ 8 files changed, 467 insertions(+), 8 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/message.go create mode 100644 pulsaradmin/pkg/pulsar/message_id_test.go create mode 100644 pulsaradmin/pkg/pulsar/subscription.go diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index a4a4bece07..601b0a9168 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -5,6 +5,7 @@ go 1.12 require ( github.com/davecgh/go-spew v1.1.1 github.com/fatih/color v1.7.0 // indirect + github.com/golang/protobuf v1.3.1 github.com/google/go-cmp v0.3.1 // indirect github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b // indirect diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index e1ad829ad7..0c7b868f1e 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -73,6 +73,7 @@ type Client interface { Functions() Functions Tenants() Tenants Topics() Topics + Subscriptions() Subscriptions Sources() Sources Sinks() Sinks Namespaces() Namespaces diff --git a/pulsaradmin/pkg/pulsar/message.go b/pulsaradmin/pkg/pulsar/message.go new file mode 100644 index 0000000000..0e364344c6 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/message.go @@ -0,0 +1,88 @@ +// 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 pulsar + +import "github.com/golang/protobuf/proto" + +type Message struct { + messageID MessageID + payload []byte + topic string + properties map[string]string +} + +func NewMessage(topic string, id MessageID, payload []byte, properties map[string]string) *Message { + return &Message{ + messageID: id, + payload: payload, + topic: topic, + properties: properties, + } +} + +func (m *Message) GetMessageID() MessageID { + return m.messageID +} + +func (m *Message) GetProperties() map[string]string { + return m.properties +} + +func (m *Message) GetPayload() []byte { + return m.payload +} + +// nolint +type SingleMessageMetadata struct { + Properties []*KeyValue `protobuf:"bytes,1,rep,name=properties" json:"properties,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,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=orderingKey" json:"ordering_key,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SingleMessageMetadata) Reset() { *m = SingleMessageMetadata{} } +func (m *SingleMessageMetadata) String() string { return proto.CompactTextString(m) } +func (*SingleMessageMetadata) ProtoMessage() {} +func (m *SingleMessageMetadata) GetPayloadSize() int32 { + if m != nil && m.PayloadSize != nil { + return *m.PayloadSize + } + return 0 +} + +// nolint +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"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *KeyValue) Reset() { *m = KeyValue{} } +func (m *KeyValue) String() string { return proto.CompactTextString(m) } +func (*KeyValue) ProtoMessage() {} diff --git a/pulsaradmin/pkg/pulsar/message_id.go b/pulsaradmin/pkg/pulsar/message_id.go index 42e3fc7f3c..898f15b949 100644 --- a/pulsaradmin/pkg/pulsar/message_id.go +++ b/pulsaradmin/pkg/pulsar/message_id.go @@ -17,8 +17,66 @@ package pulsar +import ( + "strconv" + "strings" + + "github.com/pkg/errors" +) + type MessageID struct { LedgerID int64 `json:"ledgerId"` EntryID int64 `json:"entryId"` PartitionedIndex int `json:"partitionedIndex"` + BatchIndex int `json:"-"` +} + +var Latest = MessageID{0x7fffffffffffffff, 0x7fffffffffffffff, -1, -1} +var Earliest = MessageID{-1, -1, -1, -1} + +func ParseMessageID(str string) (*MessageID, error) { + s := strings.Split(str, ":") + + m := Earliest + + if len(s) < 2 || len(s) > 4 { + return nil, errors.Errorf("invalid message id string. %s", str) + } + + ledgerID, err := strconv.ParseInt(s[0], 10, 64) + if err != nil { + return nil, errors.Errorf("invalid ledger id. %s", str) + } + m.LedgerID = ledgerID + + entryID, err := strconv.ParseInt(s[1], 10, 64) + if err != nil { + return nil, errors.Errorf("invalid entry id. %s", str) + } + m.EntryID = entryID + + if len(s) > 2 { + pi, err := strconv.Atoi(s[2]) + if err != nil { + return nil, errors.Errorf("invalid partition index. %s", str) + } + m.PartitionedIndex = pi + } + + if len(s) == 4 { + bi, err := strconv.Atoi(s[3]) + if err != nil { + return nil, errors.Errorf("invalid batch index. %s", str) + } + m.BatchIndex = bi + } + + return &m, nil +} + +func (m MessageID) String() string { + return strconv.FormatInt(m.LedgerID, 10) + ":" + + strconv.FormatInt(m.EntryID, 10) + ":" + + strconv.Itoa(m.PartitionedIndex) + ":" + + strconv.Itoa(m.BatchIndex) } diff --git a/pulsaradmin/pkg/pulsar/message_id_test.go b/pulsaradmin/pkg/pulsar/message_id_test.go new file mode 100644 index 0000000000..46c76b7423 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/message_id_test.go @@ -0,0 +1,65 @@ +// 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 pulsar + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestParseMessageId(t *testing.T) { + id, err := ParseMessageID("1:1") + assert.Nil(t, err) + assert.Equal(t, MessageID{LedgerID: 1, EntryID: 1, PartitionedIndex: -1, BatchIndex: -1}, *id) + + id, err = ParseMessageID("1:2:3") + assert.Nil(t, err) + assert.Equal(t, MessageID{LedgerID: 1, EntryID: 2, PartitionedIndex: 3, BatchIndex: -1}, *id) + + id, err = ParseMessageID("1:2:3:4") + assert.Nil(t, err) + assert.Equal(t, MessageID{LedgerID: 1, EntryID: 2, PartitionedIndex: 3, BatchIndex: 4}, *id) +} + +func TestParseMessageIdErrors(t *testing.T) { + id, err := ParseMessageID("1;1") + assert.Nil(t, id) + assert.NotNil(t, err) + assert.Equal(t, "invalid message id string. 1;1", err.Error()) + + id, err = ParseMessageID("a:1") + assert.Nil(t, id) + assert.NotNil(t, err) + assert.Equal(t, "invalid ledger id. a:1", err.Error()) + + id, err = ParseMessageID("1:a") + assert.Nil(t, id) + assert.NotNil(t, err) + assert.Equal(t, "invalid entry id. 1:a", err.Error()) + + id, err = ParseMessageID("1:2:a") + assert.Nil(t, id) + assert.NotNil(t, err) + assert.Equal(t, "invalid partition index. 1:2:a", err.Error()) + + id, err = ParseMessageID("1:2:3:a") + assert.Nil(t, id) + assert.NotNil(t, err) + assert.Equal(t, "invalid batch index. 1:2:3:a", err.Error()) +} diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index a1f96efcfe..9ec2700a0a 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -144,7 +144,7 @@ type Namespaces interface { DeleteNamespaceAntiAffinityGroup(namespace string) error // Set the deduplication status for all topics within a namespace - // When deduplication is enabled, the broker will prevent to store the same message multiple times + // When deduplication is enabled, the broker will prevent to store the same Message multiple times SetDeduplicationStatus(namespace string, enableDeduplication bool) error // Set the persistence configuration for all the topics on a namespace @@ -202,19 +202,19 @@ type Namespaces interface { // Clear backlog for all topics on a namespace ClearNamespaceBacklog(namespace NameSpaceName) error - // Set replicator-message-dispatch-rate (Replicators under this namespace + // Set replicator-Message-dispatch-rate (Replicators under this namespace // can dispatch this many messages per second) SetReplicatorDispatchRate(namespace NameSpaceName, rate DispatchRate) error - // Get replicator-message-dispatch-rate (Replicators under this namespace + // Get replicator-Message-dispatch-rate (Replicators under this namespace // can dispatch this many messages per second) GetReplicatorDispatchRate(namespace NameSpaceName) (DispatchRate, error) - // Set subscription-message-dispatch-rate (subscriptions under this namespace + // Set subscription-Message-dispatch-rate (subscriptions under this namespace // can dispatch this many messages per second) SetSubscriptionDispatchRate(namespace NameSpaceName, rate DispatchRate) error - // Get subscription-message-dispatch-rate (subscriptions under this namespace + // Get subscription-Message-dispatch-rate (subscriptions under this namespace // can dispatch this many messages per second) GetSubscriptionDispatchRate(namespace NameSpaceName) (DispatchRate, error) @@ -224,10 +224,10 @@ type Namespaces interface { // Get namespace-subscribe-rate (topics under this namespace allow subscribe times per consumer in a period) GetSubscribeRate(namespace NameSpaceName) (SubscribeRate, error) - // Set message-dispatch-rate (topics under this namespace can dispatch this many messages per second) + // Set Message-dispatch-rate (topics under this namespace can dispatch this many messages per second) SetDispatchRate(namespace NameSpaceName, rate DispatchRate) error - // Get message-dispatch-rate (topics under this namespace can dispatch this many messages per second) + // Get Message-dispatch-rate (topics under this namespace can dispatch this many messages per second) GetDispatchRate(namespace NameSpaceName) (DispatchRate, error) } diff --git a/pulsaradmin/pkg/pulsar/sink_status.go b/pulsaradmin/pkg/pulsar/sink_status.go index 68fa27dfb2..5629914ecc 100644 --- a/pulsaradmin/pkg/pulsar/sink_status.go +++ b/pulsaradmin/pkg/pulsar/sink_status.go @@ -60,7 +60,7 @@ type SinkInstanceStatusData struct { // Number of messages written to sink NumWrittenToSink int64 `json:"numWrittenToSink"` - // When was the last time we received a message from Pulsar + // When was the last time we received a Message from Pulsar LastReceivedTime int64 `json:"lastReceivedTime"` WorkerID string `json:"workerId"` diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/pulsar/subscription.go new file mode 100644 index 0000000000..8b8e4d1438 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/subscription.go @@ -0,0 +1,246 @@ +// 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 pulsar + +import ( + "bytes" + "encoding/binary" + "io" + "io/ioutil" + "net/http" + "net/url" + "strconv" + "strings" + + "github.com/golang/protobuf/proto" +) + +type Subscriptions interface { + Create(TopicName, string, MessageID) error + Delete(TopicName, string) error + List(TopicName) ([]string, error) + ResetCursorToMessageID(TopicName, string, MessageID) error + ResetCursorToTimestamp(TopicName, string, int64) error + ClearBacklog(TopicName, string) error + SkipMessages(TopicName, string, int64) error + ExpireMessages(TopicName, string, int64) error + ExpireAllMessages(TopicName, int64) error + PeekMessages(TopicName, string, int) ([]*Message, error) +} + +type subscriptions struct { + client *client + basePath string + SubPath string +} + +func (c *client) Subscriptions() Subscriptions { + return &subscriptions{ + client: c, + basePath: "", + SubPath: "subscription", + } +} + +func (s *subscriptions) Create(topic TopicName, sName string, messageID MessageID) error { + endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName)) + return s.client.put(endpoint, messageID) +} + +func (s *subscriptions) Delete(topic TopicName, sName string) error { + endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName)) + return s.client.delete(endpoint) +} + +func (s *subscriptions) List(topic TopicName) ([]string, error) { + endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), "subscriptions") + var list []string + return list, s.client.get(endpoint, &list) +} + +func (s *subscriptions) ResetCursorToMessageID(topic TopicName, sName string, id MessageID) error { + endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "resetcursor") + return s.client.post(endpoint, id) +} + +func (s *subscriptions) ResetCursorToTimestamp(topic TopicName, sName string, timestamp int64) error { + endpoint := s.client.endpoint( + s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), + "resetcursor", strconv.FormatInt(timestamp, 10)) + return s.client.post(endpoint, "") +} + +func (s *subscriptions) ClearBacklog(topic TopicName, sName string) error { + endpoint := s.client.endpoint( + s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "skip_all") + return s.client.post(endpoint, "") +} + +func (s *subscriptions) SkipMessages(topic TopicName, sName string, n int64) error { + endpoint := s.client.endpoint( + s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), + "skip", strconv.FormatInt(n, 10)) + return s.client.post(endpoint, "") +} + +func (s *subscriptions) ExpireMessages(topic TopicName, sName string, expire int64) error { + endpoint := s.client.endpoint( + s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), + "expireMessages", strconv.FormatInt(expire, 10)) + return s.client.post(endpoint, "") +} + +func (s *subscriptions) ExpireAllMessages(topic TopicName, expire int64) error { + endpoint := s.client.endpoint( + s.basePath, topic.GetRestPath(), "all_subscription", + "expireMessages", strconv.FormatInt(expire, 10)) + return s.client.post(endpoint, "") +} + +func (s *subscriptions) PeekMessages(topic TopicName, sName string, n int) ([]*Message, error) { + var msgs []*Message + + count := 1 + for n > 0 { + m, err := s.peekNthMessage(topic, sName, count) + if err != nil { + return nil, err + } + msgs = append(msgs, m...) + n -= len(m) + count++ + } + + return msgs, nil +} + +func (s *subscriptions) peekNthMessage(topic TopicName, sName string, pos int) ([]*Message, error) { + endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), "subscription", url.QueryEscape(sName), + "position", strconv.Itoa(pos)) + req, err := s.client.newRequest(http.MethodGet, endpoint) + if err != nil { + return nil, err + } + + resp, err := checkSuccessful(s.client.doRequest(req)) + if err != nil { + return nil, err + } + defer safeRespClose(resp) + + return handleResp(topic, resp) +} + +const ( + PublishTimeHeader = "X-Pulsar-Publish-Time" + BatchHeader = "X-Pulsar-Num-Batch-Message" + PropertyPrefix = "X-Pulsar-PROPERTY-" +) + +func handleResp(topic TopicName, resp *http.Response) ([]*Message, error) { + msgID := resp.Header.Get("X-Pulsar-Message-ID") + ID, err := ParseMessageID(msgID) + if err != nil { + return nil, err + } + + // read data + payload, err := ioutil.ReadAll(resp.Body) + if err != nil { + return nil, err + } + + properties := make(map[string]string) + for k := range resp.Header { + switch { + case k == PublishTimeHeader: + h := resp.Header.Get(k) + if h != "" { + properties["publish-time"] = h + } + case k == BatchHeader: + h := resp.Header.Get(k) + if h != "" { + properties[BatchHeader] = h + } + return getIndividualMsgsFromBatch(topic, ID, payload, properties) + case strings.Contains(k, PropertyPrefix): + key := strings.TrimPrefix(k, PropertyPrefix) + properties[key] = resp.Header.Get(k) + } + } + + return []*Message{NewMessage(topic.String(), *ID, payload, properties)}, nil +} + +func getIndividualMsgsFromBatch(topic TopicName, msgID *MessageID, data []byte, + properties map[string]string) ([]*Message, error) { + + batchSize, err := strconv.Atoi(properties[BatchHeader]) + if err != nil { + return nil, nil + } + + msgs := make([]*Message, 0, batchSize) + + // read all messages in batch + buf32 := make([]byte, 4) + rdBuf := bytes.NewReader(data) + for i := 0; i < batchSize; i++ { + msgID.BatchIndex = i + // singleMetaSize + if _, err := io.ReadFull(rdBuf, buf32); err != nil { + return nil, err + } + singleMetaSize := binary.BigEndian.Uint32(buf32) + + // singleMeta + singleMetaBuf := make([]byte, singleMetaSize) + if _, err := io.ReadFull(rdBuf, singleMetaBuf); err != nil { + return nil, err + } + + singleMeta := new(SingleMessageMetadata) + if err := proto.Unmarshal(singleMetaBuf, singleMeta); err != nil { + return nil, err + } + + if len(singleMeta.Properties) > 0 { + for _, v := range singleMeta.Properties { + k := *v.Key + property := *v.Value + properties[k] = property + } + } + + //payload + singlePayload := make([]byte, singleMeta.GetPayloadSize()) + if _, err := io.ReadFull(rdBuf, singlePayload); err != nil { + return nil, err + } + + msgs = append(msgs, &Message{ + topic: topic.String(), + messageID: *msgID, + payload: singlePayload, + properties: properties, + }) + } + + return msgs, nil +} From d107955936f40234e19ae8a894562361ce2c71d2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Wed, 16 Oct 2019 13:48:25 +0800 Subject: [PATCH 133/348] Add broker-stats commands for pulsarctl (streamnative/pulsarctl#84) Add broker-stats commands for pulsarctl: - allocator-stats - topics - mbeans - monitoring-metrics - load-report --- pulsaradmin/pkg/pulsar/admin.go | 1 + pulsaradmin/pkg/pulsar/allocator_stats.go | 72 ++++++++ pulsaradmin/pkg/pulsar/broker_stats.go | 97 +++++++++++ pulsaradmin/pkg/pulsar/data.go | 4 + pulsaradmin/pkg/pulsar/load_manage_report.go | 156 ++++++++++++++++++ pulsaradmin/pkg/pulsar/metrics.go | 30 ++++ pulsaradmin/pkg/pulsar/topics_stats_stream.go | 24 +++ 7 files changed, 384 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/allocator_stats.go create mode 100644 pulsaradmin/pkg/pulsar/broker_stats.go create mode 100644 pulsaradmin/pkg/pulsar/load_manage_report.go create mode 100644 pulsaradmin/pkg/pulsar/metrics.go create mode 100644 pulsaradmin/pkg/pulsar/topics_stats_stream.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 0c7b868f1e..6af5eeba12 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -78,6 +78,7 @@ type Client interface { Sinks() Sinks Namespaces() Namespaces Schemas() Schema + BrokerStats() BrokerStats } type client struct { diff --git a/pulsaradmin/pkg/pulsar/allocator_stats.go b/pulsaradmin/pkg/pulsar/allocator_stats.go new file mode 100644 index 0000000000..54c2d6a3cb --- /dev/null +++ b/pulsaradmin/pkg/pulsar/allocator_stats.go @@ -0,0 +1,72 @@ +// 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 pulsar + +type AllocatorStats struct { + NumDirectArenas int `json:"numDirectArenas"` + NumHeapArenas int `json:"numHeapArenas"` + NumThreadLocalCaches int `json:"numThreadLocalCaches"` + NormalCacheSize int `json:"normalCacheSize"` + SmallCacheSize int `json:"smallCacheSize"` + TinyCacheSize int `json:"tinyCacheSize"` + DirectArenas []PoolArenaStats `json:"directArenas"` + HeapArenas []PoolArenaStats `json:"heapArenas"` +} + +type PoolArenaStats struct { + NumTinySubpages int `json:"numTinySubpages"` + NumSmallSubpages int `json:"numSmallSubpages"` + NumChunkLists int `json:"numChunkLists"` + TinySubpages []PoolSubpageStats `json:"tinySubpages"` + SmallSubpages []PoolSubpageStats `json:"smallSubpages"` + ChunkLists []PoolChunkListStats `json:"chunkLists"` + NumAllocations int64 `json:"numAllocations"` + NumTinyAllocations int64 `json:"numTinyAllocations"` + NumSmallAllocations int64 `json:"numSmallAllocations"` + NumNormalAllocations int64 `json:"numNormalAllocations"` + NumHugeAllocations int64 `json:"numHugeAllocations"` + NumDeallocations int64 `json:"numDeallocations"` + NumTinyDeallocations int64 `json:"numTinyDeallocations"` + NumSmallDeallocations int64 `json:"numSmallDeallocations"` + NumNormalDeallocations int64 `json:"numNormalDeallocations"` + NumHugeDeallocations int64 `json:"numHugeDeallocations"` + NumActiveAllocations int64 `json:"numActiveAllocations"` + NumActiveTinyAllocations int64 `json:"numActiveTinyAllocations"` + NumActiveSmallAllocations int64 `json:"numActiveSmallAllocations"` + NumActiveNormalAllocations int64 `json:"numActiveNormalAllocations"` + NumActiveHugeAllocations int64 `json:"numActiveHugeAllocations"` +} + +type PoolSubpageStats struct { + MaxNumElements int `json:"maxNumElements"` + NumAvailable int `json:"numAvailable"` + ElementSize int `json:"elementSize"` + PageSize int `json:"pageSize"` +} + +type PoolChunkListStats struct { + MinUsage int `json:"minUsage"` + MaxUsage int `json:"maxUsage"` + Chunks []PoolChunkStats `json:"chunks"` +} + +type PoolChunkStats struct { + Usage int `json:"usage"` + ChunkSize int `json:"chunkSize"` + FreeBytes int `json:"freeBytes"` +} diff --git a/pulsaradmin/pkg/pulsar/broker_stats.go b/pulsaradmin/pkg/pulsar/broker_stats.go new file mode 100644 index 0000000000..b28974437f --- /dev/null +++ b/pulsaradmin/pkg/pulsar/broker_stats.go @@ -0,0 +1,97 @@ +// 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 pulsar + +type BrokerStats interface { + // Returns Monitoring metrics + GetMetrics() ([]Metrics, error) + + // Requests JSON string server mbean dump + GetMBeans() ([]Metrics, error) + + // Returns JSON string topics stats + GetTopics() (string, error) + + GetLoadReport() (*LocalBrokerData, error) + + GetAllocatorStats(allocatorName string) (*AllocatorStats, error) +} + +type brokerStats struct { + client *client + basePath string +} + +func (c *client) BrokerStats() BrokerStats { + return &brokerStats{ + client: c, + basePath: "/broker-stats", + } +} + +func (bs *brokerStats) GetMetrics() ([]Metrics, error) { + endpoint := bs.client.endpoint(bs.basePath, "/metrics") + var response []Metrics + err := bs.client.get(endpoint, &response) + if err != nil { + return nil, err + } + + return response, nil +} + +func (bs *brokerStats) GetMBeans() ([]Metrics, error) { + endpoint := bs.client.endpoint(bs.basePath, "/mbeans") + var response []Metrics + err := bs.client.get(endpoint, &response) + if err != nil { + return nil, err + } + + return response, nil +} + +func (bs *brokerStats) GetTopics() (string, error) { + endpoint := bs.client.endpoint(bs.basePath, "/topics") + buf, err := bs.client.getWithQueryParams(endpoint, nil, nil, false) + if err != nil { + return "", err + } + + return string(buf), nil +} + +func (bs *brokerStats) GetLoadReport() (*LocalBrokerData, error) { + endpoint := bs.client.endpoint(bs.basePath, "/load-report") + response := NewLocalBrokerData() + err := bs.client.get(endpoint, &response) + if err != nil { + return nil, nil + } + return &response, nil +} + +func (bs *brokerStats) GetAllocatorStats(allocatorName string) (*AllocatorStats, error) { + endpoint := bs.client.endpoint(bs.basePath, "/allocator-stats", allocatorName) + var allocatorStats AllocatorStats + err := bs.client.get(endpoint, &allocatorStats) + if err != nil { + return nil, err + } + return &allocatorStats, nil +} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index ea40c19d2d..764a140c5a 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -330,3 +330,7 @@ type LookupData struct { HTTPURL string `json:"httpUrl"` HTTPURLTLS string `json:"httpUrlTls"` } + +type BrokerStatsData struct { + Indent bool `json:"indent"` +} diff --git a/pulsaradmin/pkg/pulsar/load_manage_report.go b/pulsaradmin/pkg/pulsar/load_manage_report.go new file mode 100644 index 0000000000..518ee67169 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/load_manage_report.go @@ -0,0 +1,156 @@ +// 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 pulsar + +import ( + "math" +) + +type LocalBrokerData struct { + // URLs to satisfy contract of ServiceLookupData (used by NamespaceService). + WebServiceURL string `json:"webServiceUrl"` + WebServiceURLTLS string `json:"webServiceUrlTls"` + PulsarServiceURL string `json:"pulsarServiceUrl"` + PulsarServiceURLTLS string `json:"pulsarServiceUrlTls"` + PersistentTopicsEnabled bool `json:"persistentTopicsEnabled"` + NonPersistentTopicsEnabled bool `json:"nonPersistentTopicsEnabled"` + + // Most recently available system resource usage. + CPU ResourceUsage `json:"cpu"` + Memory ResourceUsage `json:"memory"` + DirectMemory ResourceUsage `json:"directMemory"` + BandwidthIn ResourceUsage `json:"bandwidthIn"` + BandwidthOut ResourceUsage `json:"bandwidthOut"` + + // Message data from the most recent namespace bundle stats. + MsgThroughputIn float64 `json:"msgThroughputIn"` + MsgThroughputOut float64 `json:"msgThroughputOut"` + MsgRateIn float64 `json:"msgRateIn"` + MsgRateOut float64 `json:"msgRateOut"` + + // Timestamp of last update. + LastUpdate int64 `json:"lastUpdate"` + + // The stats given in the most recent invocation of update. + LastStats map[string]*NamespaceBundleStats `json:"lastStats"` + NumTopics int `json:"numTopics"` + NumBundles int `json:"numBundles"` + NumConsumers int `json:"numConsumers"` + NumProducers int `json:"numProducers"` + + // All bundles belonging to this broker. + Bundles []string `json:"bundles"` + + // The bundles gained since the last invocation of update. + LastBundleGains []string `json:"lastBundleGains"` + + // The bundles lost since the last invocation of update. + LastBundleLosses []string `json:"lastBundleLosses"` + + // The version string that this broker is running, obtained from the Maven build artifact in the POM + BrokerVersionString string `json:"brokerVersionString"` + + // This place-holder requires to identify correct LoadManagerReport type while deserializing + LoadReportType string `json:"loadReportType"` + + // the external protocol data advertised by protocol handlers. + Protocols map[string]string `json:"protocols"` +} + +func NewLocalBrokerData() LocalBrokerData { + lastStats := make(map[string]*NamespaceBundleStats) + lastStats[""] = NewNamespaceBundleStats() + return LocalBrokerData{ + LastStats: lastStats, + } +} + +type NamespaceBundleStats struct { + MsgRateIn float64 `json:"msgRateIn"` + MsgThroughputIn float64 `json:"msgThroughputIn"` + MsgRateOut float64 `json:"msgRateOut"` + MsgThroughputOut float64 `json:"msgThroughputOut"` + ConsumerCount int `json:"consumerCount"` + ProducerCount int `json:"producerCount"` + TopicsNum int64 `json:"topics"` + CacheSize int64 `json:"cacheSize"` + + // Consider the throughput equal if difference is less than 100 KB/s + ThroughputDifferenceThreshold float64 `json:"throughputDifferenceThreshold"` + // Consider the msgRate equal if the difference is less than 100 + MsgRateDifferenceThreshold float64 `json:"msgRateDifferenceThreshold"` + // Consider the total topics/producers/consumers equal if the difference is less than 500 + TopicConnectionDifferenceThreshold int64 `json:"topicConnectionDifferenceThreshold"` + // Consider the cache size equal if the difference is less than 100 kb + CacheSizeDifferenceThreshold int64 `json:"cacheSizeDifferenceThreshold"` +} + +func NewNamespaceBundleStats() *NamespaceBundleStats { + return &NamespaceBundleStats{ + ThroughputDifferenceThreshold: 1e5, + MsgRateDifferenceThreshold: 100, + TopicConnectionDifferenceThreshold: 500, + CacheSizeDifferenceThreshold: 100000, + } +} + +type ResourceUsage struct { + Usage float64 `json:"usage"` + Limit float64 `json:"limit"` +} + +func (ru *ResourceUsage) Reset() { + ru.Usage = -1 + ru.Limit = -1 +} + +func (ru *ResourceUsage) CompareTo(o *ResourceUsage) int { + required := o.Limit - o.Usage + available := ru.Limit - ru.Usage + return compare(required, available) +} + +func (ru *ResourceUsage) PercentUsage() float32 { + var proportion float32 + if ru.Limit > 0 { + proportion = float32(ru.Usage) / float32(ru.Limit) + } + return proportion * 100 +} + +func compare(val1, val2 float64) int { + if val1 < val2 { + return -1 + } + + if val1 < val2 { + return 1 + } + + thisBits := math.Float64bits(val1) + anotherBits := math.Float64bits(val2) + + if thisBits == anotherBits { + return 0 + } + + if thisBits < anotherBits { + return -1 + } + return 1 +} diff --git a/pulsaradmin/pkg/pulsar/metrics.go b/pulsaradmin/pkg/pulsar/metrics.go new file mode 100644 index 0000000000..042b8a983e --- /dev/null +++ b/pulsaradmin/pkg/pulsar/metrics.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 pulsar + +type Metrics struct { + Metrics map[string]interface{} `json:"metrics"` + Dimensions map[string]string `json:"dimensions"` +} + +func NewMetrics(dimensionMap map[string]string) *Metrics { + return &Metrics{ + Metrics: make(map[string]interface{}), + Dimensions: dimensionMap, + } +} diff --git a/pulsaradmin/pkg/pulsar/topics_stats_stream.go b/pulsaradmin/pkg/pulsar/topics_stats_stream.go new file mode 100644 index 0000000000..7aa701e86b --- /dev/null +++ b/pulsaradmin/pkg/pulsar/topics_stats_stream.go @@ -0,0 +1,24 @@ +// 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 pulsar + +//var TopicsMap map[string]map[string]map[string]TopicStats + +type TopicStatsStream struct { + TopicsMap map[string]map[string]map[string]TopicStats `json:"topicStatsBuf"` +} From e674a0500dc2972a6519d469f392dba5175b7a61 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Wed, 16 Oct 2019 19:39:21 +0800 Subject: [PATCH 134/348] Add broker commands for pulsarctl (streamnative/pulsarctl#87) * Add broker commands for pulsarctl - delete-dynamic-config - get-all-dynamic-config - get-internal-config - get-runtime-config - healthcheck - list - list-dynamic-config - namespaces - update-dynamic-config --- pulsaradmin/pkg/pulsar/admin.go | 1 + pulsaradmin/pkg/pulsar/brokers.go | 152 ++++++++++++++++++ pulsaradmin/pkg/pulsar/data.go | 6 + .../pkg/pulsar/internal_configuration_data.go | 25 +++ pulsaradmin/pkg/pulsar/ns_ownership_status.go | 32 ++++ 5 files changed, 216 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/brokers.go create mode 100644 pulsaradmin/pkg/pulsar/internal_configuration_data.go create mode 100644 pulsaradmin/pkg/pulsar/ns_ownership_status.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 6af5eeba12..214700abcc 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -78,6 +78,7 @@ type Client interface { Sinks() Sinks Namespaces() Namespaces Schemas() Schema + Brokers() Brokers BrokerStats() BrokerStats } diff --git a/pulsaradmin/pkg/pulsar/brokers.go b/pulsaradmin/pkg/pulsar/brokers.go new file mode 100644 index 0000000000..9c720d5486 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/brokers.go @@ -0,0 +1,152 @@ +// 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 pulsar + +import ( + "fmt" + "net/url" + "strings" +) + +type Brokers interface { + // Get the list of active brokers in the cluster. + GetActiveBrokers(cluster string) ([]string, error) + + // Get list of updatable configuration name + GetDynamicConfigurationNames() ([]string, error) + + // Get the map of owned namespaces and their status from a single broker in the cluster + GetOwnedNamespaces(cluster, brokerURL string) (map[string]NamespaceOwnershipStatus, error) + + // It updates dynamic configuration value in to Zk that triggers watch on + // brokers and all brokers can update {@link ServiceConfiguration} value locally + UpdateDynamicConfiguration(configName, configValue string) error + + // It deletes dynamic configuration value in to Zk. It will not impact current value + // in broker but next time when broker restarts, it applies value from configuration file only. + DeleteDynamicConfiguration(configName string) error + + // Get values of runtime configuration + GetRuntimeConfigurations() (map[string]string, error) + + // Get the internal configuration data + GetInternalConfigurationData() (*InternalConfigurationData, error) + + // Get values of all overridden dynamic-configs + GetAllDynamicConfigurations() (map[string]string, error) + + // Run a health check on the broker + HealthCheck() error +} + +type broker struct { + client *client + basePath string +} + +func (c *client) Brokers() Brokers { + return &broker{ + client: c, + basePath: "/brokers", + } +} + +func (b *broker) GetActiveBrokers(cluster string) ([]string, error) { + endpoint := b.client.endpoint(b.basePath, cluster) + var res []string + err := b.client.get(endpoint, &res) + if err != nil { + return nil, err + } + return res, nil +} + +func (b *broker) GetDynamicConfigurationNames() ([]string, error) { + endpoint := b.client.endpoint(b.basePath, "/configuration/") + var res []string + err := b.client.get(endpoint, &res) + if err != nil { + return nil, err + } + return res, nil +} + +func (b *broker) GetOwnedNamespaces(cluster, brokerURL string) (map[string]NamespaceOwnershipStatus, error) { + endpoint := b.client.endpoint(b.basePath, cluster, brokerURL, "ownedNamespaces") + var res map[string]NamespaceOwnershipStatus + err := b.client.get(endpoint, &res) + if err != nil { + return nil, err + } + return res, nil +} + +func (b *broker) UpdateDynamicConfiguration(configName, configValue string) error { + value := url.QueryEscape(configValue) + endpoint := b.client.endpoint(b.basePath, "/configuration/", configName, value) + return b.client.post(endpoint, nil) +} + +func (b *broker) DeleteDynamicConfiguration(configName string) error { + endpoint := b.client.endpoint(b.basePath, "/configuration/", configName) + return b.client.delete(endpoint) +} + +func (b *broker) GetRuntimeConfigurations() (map[string]string, error) { + endpoint := b.client.endpoint(b.basePath, "/configuration/", "runtime") + var res map[string]string + err := b.client.get(endpoint, &res) + if err != nil { + return nil, err + } + return res, nil +} + +func (b *broker) GetInternalConfigurationData() (*InternalConfigurationData, error) { + endpoint := b.client.endpoint(b.basePath, "/internal-configuration") + var res InternalConfigurationData + err := b.client.get(endpoint, &res) + if err != nil { + return nil, err + } + return &res, nil +} + +func (b *broker) GetAllDynamicConfigurations() (map[string]string, error) { + endpoint := b.client.endpoint(b.basePath, "/configuration/", "values") + var res map[string]string + err := b.client.get(endpoint, &res) + if err != nil { + return nil, err + } + return res, nil +} + +func (b *broker) HealthCheck() error { + endpoint := b.client.endpoint(b.basePath, "/health") + + buf, err := b.client.getWithQueryParams(endpoint, nil, nil, false) + if err != nil { + return err + } + + if !strings.EqualFold(string(buf), "ok") { + return fmt.Errorf("health check returned unexpected result: %s", string(buf)) + } + return nil +} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index 764a140c5a..ec584e4880 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -331,6 +331,12 @@ type LookupData struct { HTTPURLTLS string `json:"httpUrlTls"` } +type BrokerData struct { + URL string `json:"brokerUrl"` + ConfigName string `json:"configName"` + ConfigValue string `json:"configValue"` +} + type BrokerStatsData struct { Indent bool `json:"indent"` } diff --git a/pulsaradmin/pkg/pulsar/internal_configuration_data.go b/pulsaradmin/pkg/pulsar/internal_configuration_data.go new file mode 100644 index 0000000000..9485164e2e --- /dev/null +++ b/pulsaradmin/pkg/pulsar/internal_configuration_data.go @@ -0,0 +1,25 @@ +// 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 pulsar + +type InternalConfigurationData struct { + ZookeeperServers string `json:"zookeeperServers"` + ConfigurationStoreServers string `json:"configurationStoreServers"` + LedgersRootPath string `json:"ledgersRootPath"` + StateStorageServiceURL string `json:"stateStorageServiceUrl"` +} diff --git a/pulsaradmin/pkg/pulsar/ns_ownership_status.go b/pulsaradmin/pkg/pulsar/ns_ownership_status.go new file mode 100644 index 0000000000..6cf37b0a0e --- /dev/null +++ b/pulsaradmin/pkg/pulsar/ns_ownership_status.go @@ -0,0 +1,32 @@ +// 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 pulsar + +type NamespaceOwnershipStatus struct { + BrokerAssignment BrokerAssignment `json:"broker_assignment"` + IsControlled bool `json:"is_controlled"` + IsActive bool `json:"is_active"` +} + +type BrokerAssignment string + +const ( + Primary BrokerAssignment = "primary" + Secondary BrokerAssignment = "secondary" + Shared BrokerAssignment = "shared" +) From 8686ad4cf2946d6cd6e8b1fe82f207195cf111de Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 18 Oct 2019 14:41:48 +0800 Subject: [PATCH 135/348] Provide the token auth (streamnative/pulsarctl#91) *Modifications* - Add the token auth for the pulsarctl --- pulsaradmin/pkg/auth/token.go | 85 +++++++++++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/admin.go | 10 ++++ 2 files changed, 95 insertions(+) create mode 100644 pulsaradmin/pkg/auth/token.go diff --git a/pulsaradmin/pkg/auth/token.go b/pulsaradmin/pkg/auth/token.go new file mode 100644 index 0000000000..c928ed585c --- /dev/null +++ b/pulsaradmin/pkg/auth/token.go @@ -0,0 +1,85 @@ +// 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 auth + +import ( + "io/ioutil" + "strings" + + "github.com/pkg/errors" +) + +type TokenAuthProvider struct { + tokenSupplier func() (string, error) +} + +// NewAuthenticationTokenWithParams return a interface of Provider with string map. +func NewAuthenticationTokenWithParams(params map[string]string) (*TokenAuthProvider, error) { + switch { + case params["token"] != "": + return NewAuthenticationToken(params["token"]), nil + case params["file"] != "": + return NewAuthenticationTokenFromFile(params["file"]), nil + default: + return nil, errors.New("missing configuration for token auth") + } +} + +// NewAuthenticationToken return a interface of Provider with a string token. +func NewAuthenticationToken(token string) *TokenAuthProvider { + return &TokenAuthProvider{ + tokenSupplier: func() (string, error) { + if token == "" { + return "", errors.New("empty token credentials") + } + return token, nil + }, + } +} + +// NewAuthenticationTokenFromFile return a interface of a Provider with a string token file path. +func NewAuthenticationTokenFromFile(tokenFilePath string) *TokenAuthProvider { + return &TokenAuthProvider{ + tokenSupplier: func() (string, error) { + data, err := ioutil.ReadFile(tokenFilePath) + if err != nil { + return "", err + } + + token := strings.Trim(string(data), " \n") + if token == "" { + return "", errors.New("empty token credentials") + } + return token, nil + }, + } +} + +func (p *TokenAuthProvider) Init() error { + // Try to read certificates immediately to provide better error at startup + _, err := p.GetData() + return err +} + +func (p *TokenAuthProvider) GetData() ([]byte, error) { + t, err := p.tokenSupplier() + if err != nil { + return nil, err + } + return []byte(t), nil +} diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 214700abcc..c615c756a7 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -47,6 +47,8 @@ type Config struct { Auth *auth.TLSAuthProvider AuthParams string TLSOptions *TLSOptions + + TokenAuth *auth.TokenAuthProvider } type TLSOptions struct { @@ -92,6 +94,8 @@ type client struct { authParams string tlsOptions *TLSOptions transport *http.Transport + + tokenAuth *auth.TokenAuthProvider } // New returns a new client @@ -103,6 +107,7 @@ func New(config *Config) (Client, error) { c := &client{ apiVersion: config.APIVersion.String(), webServiceURL: config.WebServiceURL, + tokenAuth: config.TokenAuth, } if strings.HasPrefix(c.webServiceURL, "https://") { @@ -411,6 +416,11 @@ func (c *client) doRequest(r *request) (*http.Response, error) { req.Header.Set("Accept", "application/json") } + if c.tokenAuth != nil { + data, _ := c.tokenAuth.GetData() + req.Header.Set("Authorization", "Bearer "+string(data)) + } + req.Header.Set("User-Agent", c.useragent()) hc := c.httpClient From 3013a26561904b2f9bdf94000cf03523057e9430 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Fri, 18 Oct 2019 18:19:36 +0800 Subject: [PATCH 136/348] Add pulsarctl version info (streamnative/pulsarctl#95) * Add pulsarctl version info Signed-off-by: xiaolong.ran * fix release version use way Signed-off-by: xiaolong.ran --- pulsaradmin/pkg/pulsar/admin.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index c615c756a7..157cebf6ab 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -23,7 +23,6 @@ import ( "crypto/x509" "encoding/json" "errors" - "fmt" "io" "io/ioutil" "net/http" @@ -38,6 +37,8 @@ const ( DefaultWebServiceURL = "http://localhost:8080" ) +var ReleaseVersion = "None" + // Config is used to configure the admin client type Config struct { WebServiceURL string @@ -47,8 +48,7 @@ type Config struct { Auth *auth.TLSAuthProvider AuthParams string TLSOptions *TLSOptions - - TokenAuth *auth.TokenAuthProvider + TokenAuth *auth.TokenAuthProvider } type TLSOptions struct { @@ -88,6 +88,7 @@ type client struct { webServiceURL string apiVersion string httpClient *http.Client + versionInfo string // TLS config auth *auth.TLSAuthProvider @@ -107,6 +108,7 @@ func New(config *Config) (Client, error) { c := &client{ apiVersion: config.APIVersion.String(), webServiceURL: config.WebServiceURL, + versionInfo: ReleaseVersion, tokenAuth: config.TokenAuth, } @@ -397,9 +399,8 @@ func (c *client) newRequest(method, path string) (*request, error) { return req, nil } -// TODO: add pulsarctl version func (c *client) useragent() string { - return fmt.Sprintf("pulsarctl (go)") + return c.versionInfo } func (c *client) doRequest(r *request) (*http.Response, error) { From 235299279bc94d8327aa4cef5466f54a4bc7c38c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Thu, 24 Oct 2019 10:53:04 +0800 Subject: [PATCH 137/348] Add comments for admin api (streamnative/pulsarctl#104) * Add comments for admin api Signed-off-by: xiaolong.ran * fix grammar Signed-off-by: xiaolong.ran --- pulsaradmin/pkg/pulsar/broker_stats.go | 10 +- pulsaradmin/pkg/pulsar/brokers.go | 20 +-- pulsaradmin/pkg/pulsar/cluster.go | 27 +++- ...rDefinition.go => connector_definition.go} | 0 .../{consumerConfig.go => consumer_config.go} | 0 .../{functionConfg.go => function_confg.go} | 0 pulsaradmin/pkg/pulsar/functions.go | 43 +++--- pulsaradmin/pkg/pulsar/namespace.go | 143 ++++++++++-------- pulsaradmin/pkg/pulsar/schema.go | 12 +- pulsaradmin/pkg/pulsar/sinks.go | 36 ++--- .../{sourceConfig.go => source_config.go} | 0 pulsaradmin/pkg/pulsar/sources.go | 37 ++--- pulsaradmin/pkg/pulsar/subscription.go | 27 ++++ .../pkg/pulsar/{Tenant.go => tenant.go} | 11 ++ pulsaradmin/pkg/pulsar/topic.go | 52 +++++++ .../{updateOptions.go => update_options.go} | 0 .../{WindowConfing.go => window_confing.go} | 0 17 files changed, 281 insertions(+), 137 deletions(-) rename pulsaradmin/pkg/pulsar/{connectorDefinition.go => connector_definition.go} (100%) rename pulsaradmin/pkg/pulsar/{consumerConfig.go => consumer_config.go} (100%) rename pulsaradmin/pkg/pulsar/{functionConfg.go => function_confg.go} (100%) rename pulsaradmin/pkg/pulsar/{sourceConfig.go => source_config.go} (100%) rename pulsaradmin/pkg/pulsar/{Tenant.go => tenant.go} (87%) rename pulsaradmin/pkg/pulsar/{updateOptions.go => update_options.go} (100%) rename pulsaradmin/pkg/pulsar/{WindowConfing.go => window_confing.go} (100%) diff --git a/pulsaradmin/pkg/pulsar/broker_stats.go b/pulsaradmin/pkg/pulsar/broker_stats.go index b28974437f..85694829b7 100644 --- a/pulsaradmin/pkg/pulsar/broker_stats.go +++ b/pulsaradmin/pkg/pulsar/broker_stats.go @@ -17,18 +17,21 @@ package pulsar +// BrokerStats is admin interface for broker stats management type BrokerStats interface { - // Returns Monitoring metrics + // GetMetrics returns Monitoring metrics GetMetrics() ([]Metrics, error) - // Requests JSON string server mbean dump + // GetMBeans requests JSON string server mbean dump GetMBeans() ([]Metrics, error) - // Returns JSON string topics stats + // GetTopics returns JSON string topics stats GetTopics() (string, error) + // GetLoadReport returns load report of broker GetLoadReport() (*LocalBrokerData, error) + // GetAllocatorStats returns stats from broker GetAllocatorStats(allocatorName string) (*AllocatorStats, error) } @@ -37,6 +40,7 @@ type brokerStats struct { basePath string } +// BrokerStats is used to access the broker stats endpoints func (c *client) BrokerStats() BrokerStats { return &brokerStats{ client: c, diff --git a/pulsaradmin/pkg/pulsar/brokers.go b/pulsaradmin/pkg/pulsar/brokers.go index 9c720d5486..1b79af0315 100644 --- a/pulsaradmin/pkg/pulsar/brokers.go +++ b/pulsaradmin/pkg/pulsar/brokers.go @@ -23,34 +23,35 @@ import ( "strings" ) +// Brokers is admin interface for brokers management type Brokers interface { - // Get the list of active brokers in the cluster. + // GetActiveBrokers returns the list of active brokers in the cluster. GetActiveBrokers(cluster string) ([]string, error) - // Get list of updatable configuration name + // GetDynamicConfigurationNames returns list of updatable configuration name GetDynamicConfigurationNames() ([]string, error) - // Get the map of owned namespaces and their status from a single broker in the cluster + // GetOwnedNamespaces returns the map of owned namespaces and their status from a single broker in the cluster GetOwnedNamespaces(cluster, brokerURL string) (map[string]NamespaceOwnershipStatus, error) - // It updates dynamic configuration value in to Zk that triggers watch on + // UpdateDynamicConfiguration updates dynamic configuration value in to Zk that triggers watch on // brokers and all brokers can update {@link ServiceConfiguration} value locally UpdateDynamicConfiguration(configName, configValue string) error - // It deletes dynamic configuration value in to Zk. It will not impact current value + // DeleteDynamicConfiguration deletes dynamic configuration value in to Zk. It will not impact current value // in broker but next time when broker restarts, it applies value from configuration file only. DeleteDynamicConfiguration(configName string) error - // Get values of runtime configuration + // GetRuntimeConfigurations returns values of runtime configuration GetRuntimeConfigurations() (map[string]string, error) - // Get the internal configuration data + // GetInternalConfigurationData returns the internal configuration data GetInternalConfigurationData() (*InternalConfigurationData, error) - // Get values of all overridden dynamic-configs + // GetAllDynamicConfigurations returns values of all overridden dynamic-configs GetAllDynamicConfigurations() (map[string]string, error) - // Run a health check on the broker + // HealthCheck run a health check on the broker HealthCheck() error } @@ -59,6 +60,7 @@ type broker struct { basePath string } +// Brokers is used to access the brokers endpoints func (c *client) Brokers() Brokers { return &broker{ client: c, diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index 9b486825c0..08ea892819 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -17,20 +17,42 @@ package pulsar -// Clusters is used to access the cluster endpoints. - +// Clusters is admin interface for clusters management type Clusters interface { + // List returns the list of clusters List() ([]string, error) + + // Get the configuration data for the specified cluster Get(string) (ClusterData, error) + + // Create a new cluster Create(ClusterData) error + + // Delete an existing cluster Delete(string) error + + // Update the configuration for a cluster Update(ClusterData) error + + // UpdatePeerClusters updates peer cluster names. UpdatePeerClusters(string, []string) error + + // GetPeerClusters returns peer-cluster names GetPeerClusters(string) ([]string, error) + + // CreateFailureDomain creates a domain into cluster CreateFailureDomain(FailureDomainData) error + + // GetFailureDomain returns the domain registered into a cluster GetFailureDomain(clusterName, domainName string) (FailureDomainData, error) + + // ListFailureDomains returns all registered domains in cluster ListFailureDomains(string) (FailureDomainMap, error) + + // DeleteFailureDomain deletes a domain in cluster DeleteFailureDomain(FailureDomainData) error + + // UpdateFailureDomain updates a domain into cluster UpdateFailureDomain(FailureDomainData) error } @@ -39,6 +61,7 @@ type clusters struct { basePath string } +// Clusters is used to access the cluster endpoints. func (c *client) Clusters() Clusters { return &clusters{ client: c, diff --git a/pulsaradmin/pkg/pulsar/connectorDefinition.go b/pulsaradmin/pkg/pulsar/connector_definition.go similarity index 100% rename from pulsaradmin/pkg/pulsar/connectorDefinition.go rename to pulsaradmin/pkg/pulsar/connector_definition.go diff --git a/pulsaradmin/pkg/pulsar/consumerConfig.go b/pulsaradmin/pkg/pulsar/consumer_config.go similarity index 100% rename from pulsaradmin/pkg/pulsar/consumerConfig.go rename to pulsaradmin/pkg/pulsar/consumer_config.go diff --git a/pulsaradmin/pkg/pulsar/functionConfg.go b/pulsaradmin/pkg/pulsar/function_confg.go similarity index 100% rename from pulsaradmin/pkg/pulsar/functionConfg.go rename to pulsaradmin/pkg/pulsar/function_confg.go diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index 514fd8e630..127fe79ff8 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -29,11 +29,13 @@ import ( "strings" ) +// Functions is admin interface for functions management type Functions interface { - // Create a new function. + // CreateFunc create a new function. CreateFunc(data *FunctionConfig, fileName string) error - // Create a new function by providing url from which fun-pkg can be downloaded. supported url: http/file + // CreateFuncWithURL create a new function by providing url from which fun-pkg can be downloaded. + // supported url: http/file // eg: // File: file:/dir/fileName.jar // Http: http://www.repo.com/fileName.jar @@ -44,61 +46,61 @@ type Functions interface { // url from which pkg can be downloaded CreateFuncWithURL(data *FunctionConfig, pkgURL string) error - // Stop all function instances + // StopFunction stop all function instances StopFunction(tenant, namespace, name string) error - // Stop function instance + // StopFunctionWithID stop function instance StopFunctionWithID(tenant, namespace, name string, instanceID int) error - // Delete an existing function + // DeleteFunction delete an existing function DeleteFunction(tenant, namespace, name string) error - // Start all function instances + // StartFunction start all function instances StartFunction(tenant, namespace, name string) error - // Start function instance + // StartFunctionWithID start function instance StartFunctionWithID(tenant, namespace, name string, instanceID int) error - // Restart all function instances + // RestartFunction restart all function instances RestartFunction(tenant, namespace, name string) error - // Restart function instance + // RestartFunctionWithID restart function instance RestartFunctionWithID(tenant, namespace, name string, instanceID int) error - // Get the list of functions + // GetFunctions returns the list of functions GetFunctions(tenant, namespace string) ([]string, error) - // Get the configuration for the specified function + // GetFunction returns the configuration for the specified function GetFunction(tenant, namespace, name string) (FunctionConfig, error) - // Gets the current status of a function + // GetFunctionStatus returns the current status of a function GetFunctionStatus(tenant, namespace, name string) (FunctionStatus, error) - // Gets the current status of a function instance + // GetFunctionStatusWithInstanceID returns the current status of a function instance GetFunctionStatusWithInstanceID(tenant, namespace, name string, instanceID int) (FunctionInstanceStatusData, error) - // Gets the current stats of a function + // GetFunctionStats returns the current stats of a function GetFunctionStats(tenant, namespace, name string) (FunctionStats, error) - // Gets the current stats of a function instance + // GetFunctionStatsWithInstanceID gets the current stats of a function instance GetFunctionStatsWithInstanceID(tenant, namespace, name string, instanceID int) (FunctionInstanceStatsData, error) - // Fetch the current state associated with a Pulsar Function + // GetFunctionState fetch the current state associated with a Pulsar Function // // Response Example: // { "value : 12, version : 2"} GetFunctionState(tenant, namespace, name, key string) (FunctionState, error) - // Puts the given state associated with a Pulsar Function + // PutFunctionState puts the given state associated with a Pulsar Function PutFunctionState(tenant, namespace, name string, state FunctionState) error - // Triggers the function by writing to the input topic + // TriggerFunction triggers the function by writing to the input topic TriggerFunction(tenant, namespace, name, topic, triggerValue, triggerFile string) (string, error) - // Update the configuration for a function. + // UpdateFunction updates the configuration for a function. UpdateFunction(functionConfig *FunctionConfig, fileName string, updateOptions *UpdateOptions) error - // Update the configuration for a function. + // UpdateFunctionWithURL updates the configuration for a function. // // Update a function by providing url from which fun-pkg can be downloaded. supported url: http/file // eg: @@ -112,6 +114,7 @@ type functions struct { basePath string } +// Functions is used to access the functions endpoints func (c *client) Functions() Functions { return &functions{ client: c, diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index 9ec2700a0a..43375a379b 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -23,186 +23,197 @@ import ( "strings" ) +// Namespaces is admin interface for namespaces management type Namespaces interface { - // Get the list of all the namespaces for a certain tenant + // GetNamespaces returns the list of all the namespaces for a certain tenant GetNamespaces(tenant string) ([]string, error) - // Get the list of all the topics under a certain namespace + // GetTopics returns the list of all the topics under a certain namespace GetTopics(namespace string) ([]string, error) - // Get the dump all the policies specified for a namespace + // GetPolicies returns the dump all the policies specified for a namespace GetPolicies(namespace string) (*Policies, error) - // Creates a new empty namespace with no policies attached + // CreateNamespace creates a new empty namespace with no policies attached CreateNamespace(namespace string) error - // Creates a new empty namespace with no policies attached + // CreateNsWithNumBundles creates a new empty namespace with no policies attached CreateNsWithNumBundles(namespace string, numBundles int) error - // Creates a new namespace with the specified policies + // CreateNsWithPolices creates a new namespace with the specified policies CreateNsWithPolices(namespace string, polices Policies) error - // Creates a new empty namespace with no policies attached + // CreateNsWithBundlesData creates a new empty namespace with no policies attached CreateNsWithBundlesData(namespace string, bundleData *BundlesData) error - // Delete an existing namespace + // DeleteNamespace deletes an existing namespace DeleteNamespace(namespace string) error - // Delete an existing bundle in a namespace + // DeleteNamespaceBundle deletes an existing bundle in a namespace DeleteNamespaceBundle(namespace string, bundleRange string) error - // Set the messages Time to Live for all the topics within a namespace + // SetNamespaceMessageTTL sets the messages Time to Live for all the topics within a namespace SetNamespaceMessageTTL(namespace string, ttlInSeconds int) error - // Get the message TTL for a namespace + // GetNamespaceMessageTTL returns the message TTL for a namespace GetNamespaceMessageTTL(namespace string) (int, error) - // Get the retention configuration for a namespace + // GetRetention returns the retention configuration for a namespace GetRetention(namespace string) (*RetentionPolicies, error) - // Set the retention configuration for all the topics on a namespace + // SetRetention sets the retention configuration for all the topics on a namespace SetRetention(namespace string, policy RetentionPolicies) error - // Get backlog quota map on a namespace + // GetBacklogQuotaMap returns backlog quota map on a namespace GetBacklogQuotaMap(namespace string) (map[BacklogQuotaType]BacklogQuota, error) - // Set a backlog quota for all the topics on a namespace + // SetBacklogQuota sets a backlog quota for all the topics on a namespace SetBacklogQuota(namespace string, backlogQuota BacklogQuota) error - // Remove a backlog quota policy from a namespace + // RemoveBacklogQuota removes a backlog quota policy from a namespace RemoveBacklogQuota(namespace string) error - // Set schema validation enforced for namespace + // SetSchemaValidationEnforced sets schema validation enforced for namespace SetSchemaValidationEnforced(namespace NameSpaceName, schemaValidationEnforced bool) error - // Get schema validation enforced for namespace + // GetSchemaValidationEnforced returns schema validation enforced for namespace GetSchemaValidationEnforced(namespace NameSpaceName) (bool, error) - // Set the strategy used to check the a new schema provided by a producer is compatible with the current schema - // before it is installed + // SetSchemaAutoUpdateCompatibilityStrategy sets the strategy used to check the a new schema provided + // by a producer is compatible with the current schema before it is installed SetSchemaAutoUpdateCompatibilityStrategy(namespace NameSpaceName, strategy SchemaCompatibilityStrategy) error - // Get the strategy used to check the a new schema provided by a producer is compatible with the current schema - // before it is installed + // GetSchemaAutoUpdateCompatibilityStrategy returns the strategy used to check the a new schema provided + // by a producer is compatible with the current schema before it is installed GetSchemaAutoUpdateCompatibilityStrategy(namespace NameSpaceName) (SchemaCompatibilityStrategy, error) - // Clear the offload deletion lag for a namespace. + // ClearOffloadDeleteLag clears the offload deletion lag for a namespace. ClearOffloadDeleteLag(namespace NameSpaceName) error - // Set the offload deletion lag for a namespace + // SetOffloadDeleteLag sets the offload deletion lag for a namespace SetOffloadDeleteLag(namespace NameSpaceName, timeMs int64) error - // Get the offload deletion lag for a namespace, in milliseconds + // GetOffloadDeleteLag returns the offload deletion lag for a namespace, in milliseconds GetOffloadDeleteLag(namespace NameSpaceName) (int64, error) - // Set the offloadThreshold for a namespace + // SetOffloadThreshold sets the offloadThreshold for a namespace SetOffloadThreshold(namespace NameSpaceName, threshold int64) error - // Get the offloadThreshold for a namespace + // GetOffloadThreshold returns the offloadThreshold for a namespace GetOffloadThreshold(namespace NameSpaceName) (int64, error) - // Set the compactionThreshold for a namespace + // SetCompactionThreshold sets the compactionThreshold for a namespace SetCompactionThreshold(namespace NameSpaceName, threshold int64) error - // Get the compactionThreshold for a namespace + // GetCompactionThreshold returns the compactionThreshold for a namespace GetCompactionThreshold(namespace NameSpaceName) (int64, error) - // Set maxConsumersPerSubscription for a namespace. + // SetMaxConsumersPerSubscription sets maxConsumersPerSubscription for a namespace. SetMaxConsumersPerSubscription(namespace NameSpaceName, max int) error - // Get the maxConsumersPerSubscription for a namespace. + // GetMaxConsumersPerSubscription returns the maxConsumersPerSubscription for a namespace. GetMaxConsumersPerSubscription(namespace NameSpaceName) (int, error) - // Set maxConsumersPerTopic for a namespace. + // SetMaxConsumersPerTopic sets maxConsumersPerTopic for a namespace. SetMaxConsumersPerTopic(namespace NameSpaceName, max int) error - // Get the maxProducersPerTopic for a namespace. + // GetMaxConsumersPerTopic returns the maxProducersPerTopic for a namespace. GetMaxConsumersPerTopic(namespace NameSpaceName) (int, error) - // Set maxProducersPerTopic for a namespace. + // SetMaxProducersPerTopic sets maxProducersPerTopic for a namespace. SetMaxProducersPerTopic(namespace NameSpaceName, max int) error - // Get the maxProducersPerTopic for a namespace. + // GetMaxProducersPerTopic returns the maxProducersPerTopic for a namespace. GetMaxProducersPerTopic(namespace NameSpaceName) (int, error) - // Get the replication clusters for a namespace + // GetNamespaceReplicationClusters returns the replication clusters for a namespace GetNamespaceReplicationClusters(namespace string) ([]string, error) - // Set the replication clusters for a namespace + // SetNamespaceReplicationClusters returns the replication clusters for a namespace SetNamespaceReplicationClusters(namespace string, clusterIds []string) error - // Set anti-affinity group name for a namespace + // SetNamespaceAntiAffinityGroup sets anti-affinity group name for a namespace SetNamespaceAntiAffinityGroup(namespace string, namespaceAntiAffinityGroup string) error - // Get all namespaces that grouped with given anti-affinity group + // GetAntiAffinityNamespaces returns all namespaces that grouped with given anti-affinity group GetAntiAffinityNamespaces(tenant, cluster, namespaceAntiAffinityGroup string) ([]string, error) - // Get anti-affinity group name for a namespace + // GetNamespaceAntiAffinityGroup returns anti-affinity group name for a namespace GetNamespaceAntiAffinityGroup(namespace string) (string, error) - // Delete anti-affinity group name for a namespace + // DeleteNamespaceAntiAffinityGroup deletes anti-affinity group name for a namespace DeleteNamespaceAntiAffinityGroup(namespace string) error - // Set the deduplication status for all topics within a namespace + // SetDeduplicationStatus sets the deduplication status for all topics within a namespace // When deduplication is enabled, the broker will prevent to store the same Message multiple times SetDeduplicationStatus(namespace string, enableDeduplication bool) error - // Set the persistence configuration for all the topics on a namespace + // SetPersistence sets the persistence configuration for all the topics on a namespace SetPersistence(namespace string, persistence PersistencePolicies) error - // Get the persistence configuration for a namespace + // GetPersistence returns the persistence configuration for a namespace GetPersistence(namespace string) (*PersistencePolicies, error) - // Set bookie affinity group for a namespace to isolate namespace write to bookies that are + // SetBookieAffinityGroup sets bookie affinity group for a namespace to isolate namespace write to bookies that are // part of given affinity group SetBookieAffinityGroup(namespace string, bookieAffinityGroup BookieAffinityGroupData) error - // Delete bookie affinity group configured for a namespace + // DeleteBookieAffinityGroup deletes bookie affinity group configured for a namespace DeleteBookieAffinityGroup(namespace string) error - // Get bookie affinity group configured for a namespace + // GetBookieAffinityGroup returns bookie affinity group configured for a namespace GetBookieAffinityGroup(namespace string) (*BookieAffinityGroupData, error) // Unload a namespace from the current serving broker Unload(namespace string) error - // Unload namespace bundle + // UnloadNamespaceBundle unloads namespace bundle UnloadNamespaceBundle(namespace, bundle string) error - // Split namespace bundle + // SplitNamespaceBundle splits namespace bundle SplitNamespaceBundle(namespace, bundle string, unloadSplitBundles bool) error + // GetNamespacePermissions returns permissions on a namespace GetNamespacePermissions(namespace NameSpaceName) (map[string][]AuthAction, error) + + // GrantNamespacePermission grants permission on a namespace. GrantNamespacePermission(namespace NameSpaceName, role string, action []AuthAction) error + + // RevokeNamespacePermission revokes permissions on a namespace. RevokeNamespacePermission(namespace NameSpaceName, role string) error + + // GrantSubPermission grants permission to role to access subscription's admin-api GrantSubPermission(namespace NameSpaceName, sName string, roles []string) error + + // RevokeSubPermission revoke permissions on a subscription's admin-api access RevokeSubPermission(namespace NameSpaceName, sName, role string) error - // Set the given subscription auth mode on all topics on a namespace + // SetSubscriptionAuthMode sets the given subscription auth mode on all topics on a namespace SetSubscriptionAuthMode(namespace NameSpaceName, mode SubscriptionAuthMode) error - // Set the encryption required status for all topics within a namespace + // SetEncryptionRequiredStatus sets the encryption required status for all topics within a namespace SetEncryptionRequiredStatus(namespace NameSpaceName, encrypt bool) error - // Unsubscribe the given subscription on all topics on a namespace + // UnsubscribeNamespace unsubscribe the given subscription on all topics on a namespace UnsubscribeNamespace(namespace NameSpaceName, sName string) error - // Unsubscribe the given subscription on all topics on a namespace bundle + // UnsubscribeNamespaceBundle unsubscribe the given subscription on all topics on a namespace bundle UnsubscribeNamespaceBundle(namespace NameSpaceName, bundle, sName string) error - // Clear backlog for a given subscription on all topics on a namespace bundle + // ClearNamespaceBundleBacklogForSubscription clears backlog for a given subscription on all + // topics on a namespace bundle ClearNamespaceBundleBacklogForSubscription(namespace NameSpaceName, bundle, sName string) error - // Clear backlog for all topics on a namespace bundle + // ClearNamespaceBundleBacklog clears backlog for all topics on a namespace bundle ClearNamespaceBundleBacklog(namespace NameSpaceName, bundle string) error - // Clear backlog for a given subscription on all topics on a namespace + // ClearNamespaceBacklogForSubscription clears backlog for a given subscription on all topics on a namespace ClearNamespaceBacklogForSubscription(namespace NameSpaceName, sName string) error - // Clear backlog for all topics on a namespace + // ClearNamespaceBacklog clears backlog for all topics on a namespace ClearNamespaceBacklog(namespace NameSpaceName) error - // Set replicator-Message-dispatch-rate (Replicators under this namespace + // SetReplicatorDispatchRate sets replicator-Message-dispatch-rate (Replicators under this namespace // can dispatch this many messages per second) SetReplicatorDispatchRate(namespace NameSpaceName, rate DispatchRate) error @@ -210,24 +221,27 @@ type Namespaces interface { // can dispatch this many messages per second) GetReplicatorDispatchRate(namespace NameSpaceName) (DispatchRate, error) - // Set subscription-Message-dispatch-rate (subscriptions under this namespace + // SetSubscriptionDispatchRate sets subscription-Message-dispatch-rate (subscriptions under this namespace // can dispatch this many messages per second) SetSubscriptionDispatchRate(namespace NameSpaceName, rate DispatchRate) error - // Get subscription-Message-dispatch-rate (subscriptions under this namespace + // GetSubscriptionDispatchRate returns subscription-Message-dispatch-rate (subscriptions under this namespace // can dispatch this many messages per second) GetSubscriptionDispatchRate(namespace NameSpaceName) (DispatchRate, error) - // Set namespace-subscribe-rate (topics under this namespace will limit by subscribeRate) + // SetSubscribeRate sets namespace-subscribe-rate (topics under this namespace will limit by subscribeRate) SetSubscribeRate(namespace NameSpaceName, rate SubscribeRate) error - // Get namespace-subscribe-rate (topics under this namespace allow subscribe times per consumer in a period) + // GetSubscribeRate returns namespace-subscribe-rate (topics under this namespace allow subscribe + // times per consumer in a period) GetSubscribeRate(namespace NameSpaceName) (SubscribeRate, error) - // Set Message-dispatch-rate (topics under this namespace can dispatch this many messages per second) + // SetDispatchRate sets Message-dispatch-rate (topics under this namespace can dispatch + // this many messages per second) SetDispatchRate(namespace NameSpaceName, rate DispatchRate) error - // Get Message-dispatch-rate (topics under this namespace can dispatch this many messages per second) + // GetDispatchRate returns Message-dispatch-rate (topics under this namespace can dispatch + // this many messages per second) GetDispatchRate(namespace NameSpaceName) (DispatchRate, error) } @@ -236,6 +250,7 @@ type namespaces struct { basePath string } +// Namespaces is used to access the namespaces endpoints func (c *client) Namespaces() Namespaces { return &namespaces{ client: c, diff --git a/pulsaradmin/pkg/pulsar/schema.go b/pulsaradmin/pkg/pulsar/schema.go index 880992e739..9fbd7cbb4c 100644 --- a/pulsaradmin/pkg/pulsar/schema.go +++ b/pulsaradmin/pkg/pulsar/schema.go @@ -22,20 +22,21 @@ import ( "strconv" ) +// Schema is admin interface for schema management type Schema interface { - // Retrieve the latest schema of a topic + // GetSchemaInfo retrieves the latest schema of a topic GetSchemaInfo(topic string) (*SchemaInfo, error) - // Retrieve the latest schema with version of a topic + // GetSchemaInfoWithVersion retrieves the latest schema with version of a topic GetSchemaInfoWithVersion(topic string) (*SchemaInfoWithVersion, error) - // Retrieve the schema of a topic at a given version + // GetSchemaInfoByVersion retrieves the schema of a topic at a given version GetSchemaInfoByVersion(topic string, version int64) (*SchemaInfo, error) - // Delete the schema associated with a given topic + // DeleteSchema deletes the schema associated with a given topic DeleteSchema(topic string) error - // Create a schema for a given topic + // CreateSchemaByPayload creates a schema for a given topic CreateSchemaByPayload(topic string, schemaPayload PostSchemaPayload) error } @@ -44,6 +45,7 @@ type schemas struct { basePath string } +// Schemas is used to access the schemas endpoints func (c *client) Schemas() Schema { return &schemas{ client: c, diff --git a/pulsaradmin/pkg/pulsar/sinks.go b/pulsaradmin/pkg/pulsar/sinks.go index 936bd4aceb..22b8cb454d 100644 --- a/pulsaradmin/pkg/pulsar/sinks.go +++ b/pulsaradmin/pkg/pulsar/sinks.go @@ -29,56 +29,57 @@ import ( "strings" ) +// Sinks is admin interface for sinks management type Sinks interface { - // Get the list of all the Pulsar Sinks. + // ListSinks returns the list of all the Pulsar Sinks. ListSinks(tenant, namespace string) ([]string, error) - // Get the configuration for the specified sink + // GetSink returns the configuration for the specified sink GetSink(tenant, namespace, Sink string) (SinkConfig, error) - // Create a new sink + // CreateSink creates a new sink CreateSink(config *SinkConfig, fileName string) error - // Create a new sink by providing url from which fun-pkg can be downloaded. supported url: http/file + // CreateSinkWithURL creates a new sink by providing url from which fun-pkg can be downloaded. supported url: http/file CreateSinkWithURL(config *SinkConfig, pkgURL string) error - // Update the configuration for a sink. + // UpdateSink updates the configuration for a sink. UpdateSink(config *SinkConfig, fileName string, options *UpdateOptions) error - // Update a sink by providing url from which fun-pkg can be downloaded. supported url: http/file + // UpdateSinkWithURL updates a sink by providing url from which fun-pkg can be downloaded. supported url: http/file UpdateSinkWithURL(config *SinkConfig, pkgURL string, options *UpdateOptions) error - // Delete an existing sink + // DeleteSink deletes an existing sink DeleteSink(tenant, namespace, Sink string) error - // Gets the current status of a sink. + // GetSinkStatus returns the current status of a sink. GetSinkStatus(tenant, namespace, Sink string) (SinkStatus, error) - // Gets the current status of a sink instance. + // GetSinkStatusWithID returns the current status of a sink instance. GetSinkStatusWithID(tenant, namespace, Sink string, id int) (SinkInstanceStatusData, error) - // Restart all sink instances + // RestartSink restarts all sink instances RestartSink(tenant, namespace, Sink string) error - // Restart sink instance + // RestartSinkWithID restarts sink instance RestartSinkWithID(tenant, namespace, Sink string, id int) error - // Stop all sink instances + // StopSink stops all sink instances StopSink(tenant, namespace, Sink string) error - // Stop sink instance + // StopSinkWithID stops sink instance StopSinkWithID(tenant, namespace, Sink string, id int) error - // Start all sink instances + // StartSink starts all sink instances StartSink(tenant, namespace, Sink string) error - // Start sink instance + // StartSinkWithID starts sink instance StartSinkWithID(tenant, namespace, Sink string, id int) error - // Fetches a list of supported Pulsar IO sinks currently running in cluster mode + // GetBuiltInSinks fetches a list of supported Pulsar IO sinks currently running in cluster mode GetBuiltInSinks() ([]*ConnectorDefinition, error) - // Reload the available built-in connectors, include Source and Sink + // ReloadBuiltInSinks reload the available built-in connectors, include Source and Sink ReloadBuiltInSinks() error } @@ -87,6 +88,7 @@ type sinks struct { basePath string } +// Sinks is used to access the sinks endpoints func (c *client) Sinks() Sinks { return &sinks{ client: c, diff --git a/pulsaradmin/pkg/pulsar/sourceConfig.go b/pulsaradmin/pkg/pulsar/source_config.go similarity index 100% rename from pulsaradmin/pkg/pulsar/sourceConfig.go rename to pulsaradmin/pkg/pulsar/source_config.go diff --git a/pulsaradmin/pkg/pulsar/sources.go b/pulsaradmin/pkg/pulsar/sources.go index ece4b5468a..4a823b8d6c 100644 --- a/pulsaradmin/pkg/pulsar/sources.go +++ b/pulsaradmin/pkg/pulsar/sources.go @@ -29,56 +29,58 @@ import ( "strings" ) +// Sources is admin interface for sources management type Sources interface { - // Get the list of all the Pulsar Sources. + // ListSources returns the list of all the Pulsar Sources. ListSources(tenant, namespace string) ([]string, error) - // Get the configuration for the specified source + // GetSource return the configuration for the specified source GetSource(tenant, namespace, source string) (SourceConfig, error) - // Create a new source + // CreateSource creates a new source CreateSource(config *SourceConfig, fileName string) error - // Create a new source by providing url from which fun-pkg can be downloaded. supported url: http/file + // CreateSourceWithURL creates a new source by providing url from which fun-pkg can be downloaded. + // supported url: http/file CreateSourceWithURL(config *SourceConfig, pkgURL string) error - // Update the configuration for a source. + // UpdateSource updates the configuration for a source. UpdateSource(config *SourceConfig, fileName string, options *UpdateOptions) error - // Update a source by providing url from which fun-pkg can be downloaded. supported url: http/file + // UpdateSourceWithURL updates a source by providing url from which fun-pkg can be downloaded. supported url: http/file UpdateSourceWithURL(config *SourceConfig, pkgURL string, options *UpdateOptions) error - // Delete an existing source + // DeleteSource deletes an existing source DeleteSource(tenant, namespace, source string) error - // Gets the current status of a source. + // GetSourceStatus returns the current status of a source. GetSourceStatus(tenant, namespace, source string) (SourceStatus, error) - // Gets the current status of a source instance. + // GetSourceStatusWithID returns the current status of a source instance. GetSourceStatusWithID(tenant, namespace, source string, id int) (SourceInstanceStatusData, error) - // Restart all source instances + // RestartSource restarts all source instances RestartSource(tenant, namespace, source string) error - // Restart source instance + // RestartSourceWithID restarts source instance RestartSourceWithID(tenant, namespace, source string, id int) error - // Stop all source instances + // StopSource stops all source instances StopSource(tenant, namespace, source string) error - // Stop source instance + // StopSourceWithID stops source instance StopSourceWithID(tenant, namespace, source string, id int) error - // Start all source instances + // StartSource starts all source instances StartSource(tenant, namespace, source string) error - // Start source instance + // StartSourceWithID starts source instance StartSourceWithID(tenant, namespace, source string, id int) error - // Fetches a list of supported Pulsar IO sources currently running in cluster mode + // GetBuiltInSources fetches a list of supported Pulsar IO sources currently running in cluster mode GetBuiltInSources() ([]*ConnectorDefinition, error) - // Reload the available built-in connectors, include Source and Sink + // ReloadBuiltInSources reloads the available built-in connectors, include Source and Sink ReloadBuiltInSources() error } @@ -87,6 +89,7 @@ type sources struct { basePath string } +// Sources is used to access the sources endpoints func (c *client) Sources() Sources { return &sources{ client: c, diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/pulsar/subscription.go index 8b8e4d1438..76d62d6f61 100644 --- a/pulsaradmin/pkg/pulsar/subscription.go +++ b/pulsaradmin/pkg/pulsar/subscription.go @@ -30,16 +30,42 @@ import ( "github.com/golang/protobuf/proto" ) +// Subscriptions is admin interface for subscriptions management type Subscriptions interface { + // Create a new subscription on a topic Create(TopicName, string, MessageID) error + + // Delete a subscription. + // Delete a persistent subscription from a topic. There should not be any active consumers on the subscription Delete(TopicName, string) error + + // List returns the list of subscriptions List(TopicName) ([]string, error) + + // ResetCursorToMessageID resets cursor position on a topic subscription + // @param + // messageID reset subscription to messageId (or previous nearest messageId if given messageId is not valid) ResetCursorToMessageID(TopicName, string, MessageID) error + + // ResetCursorToTimestamp resets cursor position on a topic subscription + // @param + // time reset subscription to position closest to time in ms since epoch ResetCursorToTimestamp(TopicName, string, int64) error + + // ClearBacklog skips all messages on a topic subscription ClearBacklog(TopicName, string) error + + // SkipMessages skips messages on a topic subscription SkipMessages(TopicName, string, int64) error + + // ExpireMessages expires all messages older than given N (expireTimeInSeconds) seconds for a given subscription ExpireMessages(TopicName, string, int64) error + + // ExpireAllMessages expires all messages older than given N (expireTimeInSeconds) seconds for all + // subscriptions of the persistent-topic ExpireAllMessages(TopicName, int64) error + + // PeekMessages peeks messages from a topic subscription PeekMessages(TopicName, string, int) ([]*Message, error) } @@ -49,6 +75,7 @@ type subscriptions struct { SubPath string } +// Subscriptions is used to access the subscriptions endpoints func (c *client) Subscriptions() Subscriptions { return &subscriptions{ client: c, diff --git a/pulsaradmin/pkg/pulsar/Tenant.go b/pulsaradmin/pkg/pulsar/tenant.go similarity index 87% rename from pulsaradmin/pkg/pulsar/Tenant.go rename to pulsaradmin/pkg/pulsar/tenant.go index 53d9ba5578..78ce1a174f 100644 --- a/pulsaradmin/pkg/pulsar/Tenant.go +++ b/pulsaradmin/pkg/pulsar/tenant.go @@ -17,11 +17,21 @@ package pulsar +// Tenants is admin interface for tenants management type Tenants interface { + // Create a new tenant Create(TenantData) error + + // Delete an existing tenant Delete(string) error + + // Update the admins for a tenant Update(TenantData) error + + //List returns the list of tenants List() ([]string, error) + + // Get returns the config of the tenant. Get(string) (TenantData, error) } @@ -30,6 +40,7 @@ type tenants struct { basePath string } +// Tenants is used to access the tenants endpoints func (c *client) Tenants() Tenants { return &tenants{ client: c, diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index ec00a43532..9897256780 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -22,27 +22,78 @@ import ( "strconv" ) +// Topics is admin interface for topics management type Topics interface { + // Create a topic Create(TopicName, int) error + + // Delete a topic Delete(TopicName, bool, bool) error + + // Update number of partitions of a non-global partitioned topic + // It requires partitioned-topic to be already exist and number of new partitions must be greater than existing + // number of partitions. Decrementing number of partitions requires deletion of topic which is not supported. Update(TopicName, int) error + + // GetMetadata returns metadata of a partitioned topic GetMetadata(TopicName) (PartitionedTopicMetadata, error) + + // List returns the list of topics under a namespace List(NameSpaceName) ([]string, []string, error) + + // GetInternalInfo returns the internal metadata info for the topic GetInternalInfo(TopicName) (ManagedLedgerInfo, error) + + // GetPermissions returns permissions on a topic + // Retrieve the effective permissions for a topic. These permissions are defined by the permissions set at the + // namespace level combined (union) with any eventual specific permission set on the topic. GetPermissions(TopicName) (map[string][]AuthAction, error) + + // GrantPermission grants a new permission to a client role on a single topic GrantPermission(TopicName, string, []AuthAction) error + + // RevokePermission revokes permissions to a client role on a single topic. If the permission + // was not set at the topic level, but rather at the namespace level, this operation will + // return an error (HTTP status code 412). RevokePermission(TopicName, string) error + + // Lookup a topic returns the broker URL that serves the topic Lookup(TopicName) (LookupData, error) + + // GetBundleRange returns a bundle range of a topic GetBundleRange(TopicName) (string, error) + + // GetLastMessageID returns the last commit message Id of a topic GetLastMessageID(TopicName) (MessageID, error) + + // GetStats returns the stats for the topic + // All the rates are computed over a 1 minute window and are relative the last completed 1 minute period GetStats(TopicName) (TopicStats, error) + + // GetInternalStats returns the internal stats for the topic. GetInternalStats(TopicName) (PersistentTopicInternalStats, error) + + // GetPartitionedStats returns the stats for the partitioned topic + // All the rates are computed over a 1 minute window and are relative the last completed 1 minute period GetPartitionedStats(TopicName, bool) (PartitionedTopicStats, error) + + // Terminate the topic and prevent any more messages being published on it Terminate(TopicName) (MessageID, error) + + // Offload triggers offloading messages in topic to longterm storage Offload(TopicName, MessageID) error + + // OffloadStatus checks the status of an ongoing offloading operation for a topic OffloadStatus(TopicName) (OffloadProcessStatus, error) + + // Unload a topic Unload(TopicName) error + + // Compact triggers compaction to run for a topic. A single topic can only have one instance of compaction + // running at any time. Any attempt to trigger another will be met with a ConflictException. Compact(TopicName) error + + // CompactStatus checks the status of an ongoing compaction for a topic CompactStatus(TopicName) (LongRunningProcessStatus, error) } @@ -54,6 +105,7 @@ type topics struct { lookupPath string } +// Topics is used to access the topics endpoints func (c *client) Topics() Topics { return &topics{ client: c, diff --git a/pulsaradmin/pkg/pulsar/updateOptions.go b/pulsaradmin/pkg/pulsar/update_options.go similarity index 100% rename from pulsaradmin/pkg/pulsar/updateOptions.go rename to pulsaradmin/pkg/pulsar/update_options.go diff --git a/pulsaradmin/pkg/pulsar/WindowConfing.go b/pulsaradmin/pkg/pulsar/window_confing.go similarity index 100% rename from pulsaradmin/pkg/pulsar/WindowConfing.go rename to pulsaradmin/pkg/pulsar/window_confing.go From 29fc614b7ee907e665126191445efd8685be05b3 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 24 Oct 2019 10:55:13 +0800 Subject: [PATCH 138/348] Add timeout for the http client (streamnative/pulsarctl#93) * Add timeout for the http client --- *Motivation* There is not requests timeout for the http client. We need to config a timeout for the request. *Modifications* - Set the http request timeout to 10 sencond - Set the tls handshake tiemout to 5 sencond --- pulsaradmin/pkg/pulsar/admin.go | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 157cebf6ab..263aa21cc9 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -29,12 +29,14 @@ import ( "net/url" "path" "strings" + "time" "github.com/streamnative/pulsar-admin-go/pkg/auth" ) const ( - DefaultWebServiceURL = "http://localhost:8080" + DefaultWebServiceURL = "http://localhost:8080" + DefaultHTTPTimeOutDuration = 5 * time.Minute ) var ReleaseVersion = "None" @@ -42,6 +44,7 @@ var ReleaseVersion = "None" // Config is used to configure the admin client type Config struct { WebServiceURL string + HTTPTimeout time.Duration HTTPClient *http.Client APIVersion APIVersion @@ -60,7 +63,9 @@ type TLSOptions struct { func DefaultConfig() *Config { config := &Config{ WebServiceURL: DefaultWebServiceURL, - HTTPClient: http.DefaultClient, + HTTPClient: &http.Client{ + Timeout: DefaultHTTPTimeOutDuration, + }, TLSOptions: &TLSOptions{ AllowInsecureConnection: false, @@ -129,6 +134,7 @@ func New(config *Config) (Client, error) { } c.transport = &http.Transport{ + TLSHandshakeTimeout: 15 * time.Second, MaxIdleConnsPerHost: 10, TLSClientConfig: tlsConf, } @@ -426,8 +432,11 @@ func (c *client) doRequest(r *request) (*http.Response, error) { hc := c.httpClient if hc == nil { - hc = http.DefaultClient + hc = &http.Client{ + Timeout: DefaultHTTPTimeOutDuration, + } } + if c.transport != nil { hc.Transport = c.transport } From 5467ad133e353db832da233f31f7340509a293f8 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 24 Oct 2019 13:15:37 +0800 Subject: [PATCH 139/348] Add `SCOPE` for topic commands (streamnative/pulsarctl#72) Master issue: streamnative/pulsarctl#55 *Motivation* Add a new type `SCOPE` to indicate the command will work for which type of topic. --- pulsaradmin/pkg/pulsar/descriptions.go | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/descriptions.go b/pulsaradmin/pkg/pulsar/descriptions.go index 44c933bafe..513e73f7d7 100644 --- a/pulsaradmin/pkg/pulsar/descriptions.go +++ b/pulsaradmin/pkg/pulsar/descriptions.go @@ -26,12 +26,14 @@ var USEDFOR = "USED FOR:" var PERMISSION = "REQUIRED PERMISSION:" var EXAMPLES = "EXAMPLES:" var OUTPUT = "OUTPUT:" +var SCOPE = "SCOPE:" type LongDescription struct { CommandUsedFor string CommandPermission string CommandExamples []Example CommandOutput []Output + CommandScope string } type Example struct { @@ -45,12 +47,19 @@ type Output struct { } func (desc *LongDescription) ToString() string { - return USEDFOR + "\n" + + out := USEDFOR + "\n" + SPACES + desc.CommandUsedFor + "\n\n" + PERMISSION + "\n" + SPACES + desc.CommandPermission + "\n\n" + OUTPUT + "\n" + desc.outputToString() + + if desc.CommandScope != "" { + out += SCOPE + "\n" + + SPACES + desc.CommandScope + "\n\n" + } + + return out } func (desc *LongDescription) ExampleToString() string { From 8e26bddbcb9713b3ee3fc53669ed30cf0a08eac1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Thu, 24 Oct 2019 13:27:12 +0800 Subject: [PATCH 140/348] Add ns-isolation-policy commands for pulsarctl (streamnative/pulsarctl#88) Add `ns-isolation-policy` commands for pulsarctl - set - get - list - delete - brokers - broker --- pulsaradmin/pkg/pulsar/admin.go | 1 + .../pkg/pulsar/broker_ns_isolation_data.go | 25 ++++ pulsaradmin/pkg/pulsar/data.go | 8 ++ pulsaradmin/pkg/pulsar/ns_isolation_data.go | 95 +++++++++++++++ pulsaradmin/pkg/pulsar/ns_isolation_policy.go | 108 ++++++++++++++++++ 5 files changed, 237 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/broker_ns_isolation_data.go create mode 100644 pulsaradmin/pkg/pulsar/ns_isolation_data.go create mode 100644 pulsaradmin/pkg/pulsar/ns_isolation_policy.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 263aa21cc9..162949d1e4 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -85,6 +85,7 @@ type Client interface { Sinks() Sinks Namespaces() Namespaces Schemas() Schema + NsIsolationPolicy() NsIsolationPolicy Brokers() Brokers BrokerStats() BrokerStats } diff --git a/pulsaradmin/pkg/pulsar/broker_ns_isolation_data.go b/pulsaradmin/pkg/pulsar/broker_ns_isolation_data.go new file mode 100644 index 0000000000..29b9ef55c5 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/broker_ns_isolation_data.go @@ -0,0 +1,25 @@ +// 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 pulsar + +type BrokerNamespaceIsolationData struct { + BrokerName string `json:"brokerName"` + PolicyName string `json:"policyName"` + IsPrimary bool `json:"isPrimary"` + NamespaceRegex []string `json:"namespaceRegex"` +} diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index ec584e4880..996045f84c 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -331,6 +331,14 @@ type LookupData struct { HTTPURLTLS string `json:"httpUrlTls"` } +type NsIsolationPoliciesData struct { + Namespaces []string `json:"namespaces"` + Primary []string `json:"primary"` + Secondary []string `json:"secondary"` + AutoFailoverPolicyTypeName string `json:"autoFailoverPolicyTypeName"` + AutoFailoverPolicyParams string `json:"autoFailoverPolicyParams"` +} + type BrokerData struct { URL string `json:"brokerUrl"` ConfigName string `json:"configName"` diff --git a/pulsaradmin/pkg/pulsar/ns_isolation_data.go b/pulsaradmin/pkg/pulsar/ns_isolation_data.go new file mode 100644 index 0000000000..bb87a1a5e2 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/ns_isolation_data.go @@ -0,0 +1,95 @@ +// 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 pulsar + +import ( + "github.com/pkg/errors" +) + +type NamespaceIsolationData struct { + Namespaces []string `json:"namespaces"` + Primary []string `json:"primary"` + Secondary []string `json:"secondary"` + AutoFailoverPolicy AutoFailoverPolicyData `json:"auto_failover_policy"` +} + +type AutoFailoverPolicyData struct { + PolicyType AutoFailoverPolicyType `json:"policy_type"` + Parameters map[string]string `json:"parameters"` +} + +type AutoFailoverPolicyType string + +const ( + MinAvailable AutoFailoverPolicyType = "min_available" +) + +func fromString(autoFailoverPolicyTypeName string) AutoFailoverPolicyType { + switch autoFailoverPolicyTypeName { + case "min_available": + return MinAvailable + default: + return "" + } +} + +func CreateNamespaceIsolationData(namespaces, primary, secondry []string, autoFailoverPolicyTypeName string, + autoFailoverPolicyParams map[string]string) (*NamespaceIsolationData, error) { + nsIsolationData := new(NamespaceIsolationData) + if len(namespaces) == 0 { + return nil, errors.New("unable to parse namespaces parameter list") + } + + if len(primary) == 0 { + return nil, errors.New("unable to parse primary parameter list") + } + + if len(secondry) == 0 { + return nil, errors.New("unable to parse secondry parameter list") + } + + nsIsolationData.Namespaces = namespaces + nsIsolationData.Primary = primary + nsIsolationData.Secondary = secondry + nsIsolationData.AutoFailoverPolicy.PolicyType = fromString(autoFailoverPolicyTypeName) + nsIsolationData.AutoFailoverPolicy.Parameters = autoFailoverPolicyParams + + // validation if necessary + if nsIsolationData.AutoFailoverPolicy.PolicyType == MinAvailable { + err := true + expectParamKeys := []string{"min_limit", "usage_threshold"} + + if len(autoFailoverPolicyParams) == len(expectParamKeys) { + for _, paramKey := range expectParamKeys { + if _, ok := autoFailoverPolicyParams[paramKey]; !ok { + break + } + } + err = false + } + + if err { + return nil, errors.Errorf("Unknown auto failover policy params specified: %v", autoFailoverPolicyParams) + } + } else { + // either we don't handle the new type or user has specified a bad type + return nil, errors.Errorf("Unknown auto failover policy type specified : %v", autoFailoverPolicyTypeName) + } + + return nsIsolationData, nil +} diff --git a/pulsaradmin/pkg/pulsar/ns_isolation_policy.go b/pulsaradmin/pkg/pulsar/ns_isolation_policy.go new file mode 100644 index 0000000000..99c573fde0 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/ns_isolation_policy.go @@ -0,0 +1,108 @@ +// 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 pulsar + +type NsIsolationPolicy interface { + // Create a namespace isolation policy for a cluster + CreateNamespaceIsolationPolicy(cluster, policyName string, namespaceIsolationData NamespaceIsolationData) error + + // Delete a namespace isolation policy for a cluster + DeleteNamespaceIsolationPolicy(cluster, policyName string) error + + // Get a single namespace isolation policy for a cluster + GetNamespaceIsolationPolicy(cluster, policyName string) (*NamespaceIsolationData, error) + + // Get the namespace isolation policies of a cluster + GetNamespaceIsolationPolicies(cluster string) (map[string]NamespaceIsolationData, error) + + // Returns list of active brokers with namespace-isolation policies attached to it. + GetBrokersWithNamespaceIsolationPolicy(cluster string) ([]BrokerNamespaceIsolationData, error) + + // Returns active broker with namespace-isolation policies attached to it. + GetBrokerWithNamespaceIsolationPolicy(cluster, broker string) (*BrokerNamespaceIsolationData, error) +} + +type nsIsolationPolicy struct { + client *client + basePath string +} + +func (c *client) NsIsolationPolicy() NsIsolationPolicy { + return &nsIsolationPolicy{ + client: c, + basePath: "/clusters", + } +} + +func (n *nsIsolationPolicy) CreateNamespaceIsolationPolicy(cluster, policyName string, + namespaceIsolationData NamespaceIsolationData) error { + return n.setNamespaceIsolationPolicy(cluster, policyName, namespaceIsolationData) +} + +func (n *nsIsolationPolicy) setNamespaceIsolationPolicy(cluster, policyName string, + namespaceIsolationData NamespaceIsolationData) error { + endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", policyName) + return n.client.post(endpoint, &namespaceIsolationData) +} + +func (n *nsIsolationPolicy) DeleteNamespaceIsolationPolicy(cluster, policyName string) error { + endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", policyName) + return n.client.delete(endpoint) +} + +func (n *nsIsolationPolicy) GetNamespaceIsolationPolicy(cluster, policyName string) (*NamespaceIsolationData, error) { + endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", policyName) + var nsIsolationData NamespaceIsolationData + err := n.client.get(endpoint, &nsIsolationData) + if err != nil { + return nil, err + } + return &nsIsolationData, nil +} + +func (n *nsIsolationPolicy) GetNamespaceIsolationPolicies(cluster string) (map[string]NamespaceIsolationData, error) { + endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies") + var tmpMap map[string]NamespaceIsolationData + err := n.client.get(endpoint, &tmpMap) + if err != nil { + return nil, err + } + return tmpMap, nil +} + +func (n *nsIsolationPolicy) GetBrokersWithNamespaceIsolationPolicy(cluster string) ( + []BrokerNamespaceIsolationData, error) { + endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", "brokers") + var res []BrokerNamespaceIsolationData + err := n.client.get(endpoint, &res) + if err != nil { + return nil, err + } + return res, nil +} + +func (n *nsIsolationPolicy) GetBrokerWithNamespaceIsolationPolicy(cluster, + broker string) (*BrokerNamespaceIsolationData, error) { + endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", "brokers", broker) + var brokerNamespaceIsolationData BrokerNamespaceIsolationData + err := n.client.get(endpoint, &brokerNamespaceIsolationData) + if err != nil { + return nil, err + } + return &brokerNamespaceIsolationData, nil +} From a4b64d6e63f0af60d9fde121e19656fc80596e06 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Thu, 24 Oct 2019 17:30:15 +0800 Subject: [PATCH 141/348] Add functions-worker commands for pulsarctl (streamnative/pulsarctl#92) Add functions-worker commands for pulsarctl: - function-stats - get-cluster - get-cluster-leader - get-function-assignments - monitoring-metrics --- pulsaradmin/pkg/pulsar/admin.go | 1 + pulsaradmin/pkg/pulsar/functions_worker.go | 99 ++++++++++++++++++++++ pulsaradmin/pkg/pulsar/worker_info.go | 29 +++++++ 3 files changed, 129 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/functions_worker.go create mode 100644 pulsaradmin/pkg/pulsar/worker_info.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 162949d1e4..7c7c7233e4 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -88,6 +88,7 @@ type Client interface { NsIsolationPolicy() NsIsolationPolicy Brokers() Brokers BrokerStats() BrokerStats + FunctionsWorker() FunctionsWorker } type client struct { diff --git a/pulsaradmin/pkg/pulsar/functions_worker.go b/pulsaradmin/pkg/pulsar/functions_worker.go new file mode 100644 index 0000000000..acdaf85c31 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/functions_worker.go @@ -0,0 +1,99 @@ +// 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 pulsar + +type FunctionsWorker interface { + // Get all functions stats on a worker + GetFunctionsStats() ([]*WorkerFunctionInstanceStats, error) + + // Get worker metrics + GetMetrics() ([]*Metrics, error) + + // Get List of all workers belonging to this cluster + GetCluster() ([]*WorkerInfo, error) + + // Get the worker who is the leader of the clusterv + GetClusterLeader() (*WorkerInfo, error) + + // Get the function assignment among the cluster + GetAssignments() (map[string][]string, error) +} + +type worker struct { + client *client + workerPath string + workerStatsPath string +} + +func (c *client) FunctionsWorker() FunctionsWorker { + return &worker{ + client: c, + workerPath: "/worker", + workerStatsPath: "/worker-stats", + } +} + +func (w *worker) GetFunctionsStats() ([]*WorkerFunctionInstanceStats, error) { + endpoint := w.client.endpoint(w.workerStatsPath, "functionsmetrics") + var workerStats []*WorkerFunctionInstanceStats + err := w.client.get(endpoint, &workerStats) + if err != nil { + return nil, err + } + return workerStats, nil +} + +func (w *worker) GetMetrics() ([]*Metrics, error) { + endpoint := w.client.endpoint(w.workerStatsPath, "metrics") + var metrics []*Metrics + err := w.client.get(endpoint, &metrics) + if err != nil { + return nil, err + } + return metrics, nil +} + +func (w *worker) GetCluster() ([]*WorkerInfo, error) { + endpoint := w.client.endpoint(w.workerPath, "cluster") + var workersInfo []*WorkerInfo + err := w.client.get(endpoint, &workersInfo) + if err != nil { + return nil, err + } + return workersInfo, nil +} + +func (w *worker) GetClusterLeader() (*WorkerInfo, error) { + endpoint := w.client.endpoint(w.workerPath, "cluster", "leader") + var workerInfo WorkerInfo + err := w.client.get(endpoint, &workerInfo) + if err != nil { + return nil, err + } + return &workerInfo, nil +} + +func (w *worker) GetAssignments() (map[string][]string, error) { + endpoint := w.client.endpoint(w.workerPath, "assignments") + var assignments map[string][]string + err := w.client.get(endpoint, &assignments) + if err != nil { + return nil, err + } + return assignments, nil +} diff --git a/pulsaradmin/pkg/pulsar/worker_info.go b/pulsaradmin/pkg/pulsar/worker_info.go new file mode 100644 index 0000000000..f98cc797ec --- /dev/null +++ b/pulsaradmin/pkg/pulsar/worker_info.go @@ -0,0 +1,29 @@ +// 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 pulsar + +type WorkerInfo struct { + WorkerID string `json:"workerId"` + WorkerHostname string `json:"workerHostname"` + Port int `json:"port"` +} + +type WorkerFunctionInstanceStats struct { + Name string `json:"name"` + Metrics FunctionInstanceStatsData `json:"metrics"` +} From 5abac5c3154a670fdb07f56c1b3934f2ce4d7722 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Thu, 24 Oct 2019 19:32:27 +0800 Subject: [PATCH 142/348] Add resource-quotas commands for pulsarctl (streamnative/pulsarctl#90) * Add resource-quotas commands for pulsarctl Signed-off-by: xiaolong.ran --- pulsaradmin/pkg/pulsar/admin.go | 1 + pulsaradmin/pkg/pulsar/data.go | 11 +++ pulsaradmin/pkg/pulsar/resource_quota.go | 44 ++++++++++++ pulsaradmin/pkg/pulsar/resource_quotas.go | 82 +++++++++++++++++++++++ 4 files changed, 138 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/resource_quota.go create mode 100644 pulsaradmin/pkg/pulsar/resource_quotas.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 7c7c7233e4..c714b0b0f6 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -88,6 +88,7 @@ type Client interface { NsIsolationPolicy() NsIsolationPolicy Brokers() Brokers BrokerStats() BrokerStats + ResourceQuotas() ResourceQuotas FunctionsWorker() FunctionsWorker } diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/data.go index 996045f84c..003ef149a3 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/data.go @@ -348,3 +348,14 @@ type BrokerData struct { type BrokerStatsData struct { Indent bool `json:"indent"` } + +type ResourceQuotaData struct { + Names string `json:"names"` + Bundle string `json:"bundle"` + MsgRateIn int64 `json:"msgRateIn"` + MsgRateOut int64 `json:"msgRateOut"` + BandwidthIn int64 `json:"bandwidthIn"` + BandwidthOut int64 `json:"bandwidthOut"` + Memory int64 `json:"memory"` + Dynamic bool `json:"dynamic"` +} diff --git a/pulsaradmin/pkg/pulsar/resource_quota.go b/pulsaradmin/pkg/pulsar/resource_quota.go new file mode 100644 index 0000000000..40bd585990 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/resource_quota.go @@ -0,0 +1,44 @@ +// 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 pulsar + +type ResourceQuota struct { + // messages published per second + MsgRateIn float64 `json:"msgRateIn"` + // messages consumed per second + MsgRateOut float64 `json:"msgRateOut"` + // incoming bytes per second + BandwidthIn float64 `json:"bandwidthIn"` + // outgoing bytes per second + BandwidthOut float64 `json:"bandwidthOut"` + // used memory in Mbytes + Memory float64 `json:"memory"` + // allow the quota be dynamically re-calculated according to real traffic + Dynamic bool `json:"dynamic"` +} + +func NewResourceQuota() *ResourceQuota { + return &ResourceQuota{ + MsgRateIn: 0.0, + MsgRateOut: 0.0, + BandwidthIn: 0.0, + BandwidthOut: 0.0, + Memory: 0.0, + Dynamic: true, + } +} diff --git a/pulsaradmin/pkg/pulsar/resource_quotas.go b/pulsaradmin/pkg/pulsar/resource_quotas.go new file mode 100644 index 0000000000..39afcc136d --- /dev/null +++ b/pulsaradmin/pkg/pulsar/resource_quotas.go @@ -0,0 +1,82 @@ +// 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 pulsar + +type ResourceQuotas interface { + // Get default resource quota for new resource bundles. + GetDefaultResourceQuota() (*ResourceQuota, error) + + // Set default resource quota for new namespace bundles. + SetDefaultResourceQuota(quota ResourceQuota) error + + // Get resource quota of a namespace bundle. + GetNamespaceBundleResourceQuota(namespace, bundle string) (*ResourceQuota, error) + + // Set resource quota for a namespace bundle. + SetNamespaceBundleResourceQuota(namespace, bundle string, quota ResourceQuota) error + + // Reset resource quota for a namespace bundle to default value. + ResetNamespaceBundleResourceQuota(namespace, bundle string) error +} + +type resource struct { + client *client + basePath string +} + +func (c *client) ResourceQuotas() ResourceQuotas { + return &resource{ + client: c, + basePath: "/resource-quotas", + } +} + +func (r *resource) GetDefaultResourceQuota() (*ResourceQuota, error) { + endpoint := r.client.endpoint(r.basePath) + var quota ResourceQuota + err := r.client.get(endpoint, "a) + if err != nil { + return nil, err + } + return "a, nil +} + +func (r *resource) SetDefaultResourceQuota(quota ResourceQuota) error { + endpoint := r.client.endpoint(r.basePath) + return r.client.post(endpoint, "a) +} + +func (r *resource) GetNamespaceBundleResourceQuota(namespace, bundle string) (*ResourceQuota, error) { + endpoint := r.client.endpoint(r.basePath, namespace, bundle) + var quota ResourceQuota + err := r.client.get(endpoint, "a) + if err != nil { + return nil, err + } + return "a, nil +} + +func (r *resource) SetNamespaceBundleResourceQuota(namespace, bundle string, quota ResourceQuota) error { + endpoint := r.client.endpoint(r.basePath, namespace, bundle) + return r.client.post(endpoint, "a) +} + +func (r *resource) ResetNamespaceBundleResourceQuota(namespace, bundle string) error { + endpoint := r.client.endpoint(r.basePath, namespace, bundle) + return r.client.delete(endpoint) +} From 2fcad0cc904b2935471ccc4f20ed742229a6ce6b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Wed, 30 Oct 2019 15:25:22 +0800 Subject: [PATCH 143/348] Refactor pulsar pkg for pulsarctl (streamnative/pulsarctl#124) * Refactor pulsar pkg for pulsarctl --- pulsaradmin/pkg/pulsar/admin.go | 10 +- pulsaradmin/pkg/pulsar/broker_stats.go | 28 +- pulsaradmin/pkg/pulsar/brokers.go | 14 +- pulsaradmin/pkg/pulsar/cluster.go | 42 +-- .../pkg/pulsar/{ => common}/api_version.go | 2 +- .../pulsar/{ => common}/api_version_test.go | 2 +- .../pkg/pulsar/{ => common}/auth_action.go | 2 +- .../pkg/pulsar/{ => common}/auth_polices.go | 2 +- .../pkg/pulsar/{ => common}/descriptions.go | 2 +- .../pulsar/{ => common}/descriptions_test.go | 2 +- pulsaradmin/pkg/pulsar/{ => common}/errors.go | 4 +- pulsaradmin/pkg/pulsar/functions.go | 63 ++-- pulsaradmin/pkg/pulsar/functions_worker.go | 28 +- pulsaradmin/pkg/pulsar/namespace.go | 290 +++++++++--------- pulsaradmin/pkg/pulsar/ns_isolation_policy.go | 36 ++- pulsaradmin/pkg/pulsar/resource_quotas.go | 24 +- pulsaradmin/pkg/pulsar/schema.go | 51 +-- pulsaradmin/pkg/pulsar/sinks.go | 42 +-- pulsaradmin/pkg/pulsar/sources.go | 44 +-- pulsaradmin/pkg/pulsar/subscription.go | 69 +++-- pulsaradmin/pkg/pulsar/tenant.go | 18 +- pulsaradmin/pkg/pulsar/topic.go | 109 +++---- .../pkg/pulsar/{ => utils}/allocator_stats.go | 2 +- .../pkg/pulsar/{ => utils}/backlog_quota.go | 2 +- .../{ => utils}/broker_ns_isolation_data.go | 2 +- .../pkg/pulsar/{ => utils}/bundles_data.go | 2 +- .../{ => utils}/connector_definition.go | 2 +- .../pkg/pulsar/{ => utils}/consumer_config.go | 2 +- pulsaradmin/pkg/pulsar/{ => utils}/data.go | 2 +- .../pkg/pulsar/{ => utils}/dispatch_rate.go | 2 +- .../pkg/pulsar/{ => utils}/function_confg.go | 2 +- .../pkg/pulsar/{ => utils}/function_state.go | 2 +- .../pkg/pulsar/{ => utils}/function_status.go | 2 +- .../pkg/pulsar/{ => utils}/functions_stats.go | 2 +- .../internal_configuration_data.go | 2 +- .../pulsar/{ => utils}/load_manage_report.go | 2 +- .../long_running_process_status.go | 2 +- pulsaradmin/pkg/pulsar/{ => utils}/message.go | 24 +- .../pkg/pulsar/{ => utils}/message_id.go | 2 +- .../pkg/pulsar/{ => utils}/message_id_test.go | 2 +- pulsaradmin/pkg/pulsar/{ => utils}/metrics.go | 2 +- .../pkg/pulsar/{ => utils}/namespace_name.go | 2 +- .../pulsar/{ => utils}/namespace_name_test.go | 2 +- .../pulsar/{ => utils}/ns_isolation_data.go | 2 +- .../pulsar/{ => utils}/ns_ownership_status.go | 2 +- .../{ => utils}/persistence_policies.go | 2 +- .../pkg/pulsar/{ => utils}/policies.go | 10 +- .../pkg/pulsar/{ => utils}/resource_quota.go | 2 +- .../pkg/pulsar/{ => utils}/resources.go | 2 +- .../pulsar/{ => utils}/retention_policies.go | 2 +- .../pkg/pulsar/{ => utils}/schema_strategy.go | 2 +- .../pkg/pulsar/{ => utils}/schema_util.go | 8 +- .../{sinkConfig.go => utils/sink_config.go} | 2 +- .../pkg/pulsar/{ => utils}/sink_status.go | 2 +- .../pkg/pulsar/{ => utils}/source_config.go | 2 +- .../pkg/pulsar/{ => utils}/source_status.go | 2 +- .../{ => utils}/subscription_auth_mode.go | 2 +- .../pkg/pulsar/{ => utils}/topic_domain.go | 2 +- .../pkg/pulsar/{ => utils}/topic_name.go | 30 +- .../pkg/pulsar/{ => utils}/topic_name_test.go | 2 +- .../pulsar/{ => utils}/topics_stats_stream.go | 2 +- .../pkg/pulsar/{ => utils}/update_options.go | 2 +- pulsaradmin/pkg/pulsar/{ => utils}/utils.go | 4 +- .../pkg/pulsar/{ => utils}/window_confing.go | 2 +- .../pkg/pulsar/{ => utils}/worker_info.go | 2 +- 65 files changed, 550 insertions(+), 486 deletions(-) rename pulsaradmin/pkg/pulsar/{ => common}/api_version.go (98%) rename pulsaradmin/pkg/pulsar/{ => common}/api_version_test.go (98%) rename pulsaradmin/pkg/pulsar/{ => common}/auth_action.go (98%) rename pulsaradmin/pkg/pulsar/{ => common}/auth_polices.go (98%) rename pulsaradmin/pkg/pulsar/{ => common}/descriptions.go (99%) rename pulsaradmin/pkg/pulsar/{ => common}/descriptions_test.go (99%) rename pulsaradmin/pkg/pulsar/{ => common}/errors.go (94%) rename pulsaradmin/pkg/pulsar/{ => utils}/allocator_stats.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/backlog_quota.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/broker_ns_isolation_data.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/bundles_data.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/connector_definition.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/consumer_config.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/data.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/dispatch_rate.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/function_confg.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/function_state.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/function_status.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/functions_stats.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/internal_configuration_data.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/load_manage_report.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/long_running_process_status.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/message.go (92%) rename pulsaradmin/pkg/pulsar/{ => utils}/message_id.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/message_id_test.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/metrics.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/namespace_name.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/namespace_name_test.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/ns_isolation_data.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/ns_ownership_status.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/persistence_policies.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/policies.go (95%) rename pulsaradmin/pkg/pulsar/{ => utils}/resource_quota.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/resources.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/retention_policies.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/schema_strategy.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/schema_util.go (91%) rename pulsaradmin/pkg/pulsar/{sinkConfig.go => utils/sink_config.go} (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/sink_status.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/source_config.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/source_status.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/subscription_auth_mode.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/topic_domain.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/topic_name.go (87%) rename pulsaradmin/pkg/pulsar/{ => utils}/topic_name_test.go (99%) rename pulsaradmin/pkg/pulsar/{ => utils}/topics_stats_stream.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/update_options.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/utils.go (91%) rename pulsaradmin/pkg/pulsar/{ => utils}/window_confing.go (98%) rename pulsaradmin/pkg/pulsar/{ => utils}/worker_info.go (98%) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index c714b0b0f6..bf481627f0 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -32,6 +32,8 @@ import ( "time" "github.com/streamnative/pulsar-admin-go/pkg/auth" + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) const ( @@ -46,7 +48,7 @@ type Config struct { WebServiceURL string HTTPTimeout time.Duration HTTPClient *http.Client - APIVersion APIVersion + APIVersion common.APIVersion Auth *auth.TLSAuthProvider AuthParams string @@ -177,7 +179,7 @@ func (c *client) getTLSConfig() (*tls.Config, error) { } func (c *client) endpoint(componentPath string, parts ...string) string { - return path.Join(makeHTTPPath(c.apiVersion, componentPath), endpoint(parts...)) + return path.Join(utils.MakeHTTPPath(c.apiVersion, componentPath), endpoint(parts...)) } // get is used to do a GET request against an endpoint @@ -473,7 +475,7 @@ func safeRespClose(resp *http.Response) { // responseError is used to parse a response into a pulsar error func responseError(resp *http.Response) error { - var e Error + var e common.Error body, err := ioutil.ReadAll(resp.Body) if err != nil { e.Reason = err.Error() @@ -486,7 +488,7 @@ func responseError(resp *http.Response) error { e.Code = resp.StatusCode if e.Reason == "" { - e.Reason = unknownErrorReason + e.Reason = common.UnknownErrorReason } return e diff --git a/pulsaradmin/pkg/pulsar/broker_stats.go b/pulsaradmin/pkg/pulsar/broker_stats.go index 85694829b7..bcf8a17ddf 100644 --- a/pulsaradmin/pkg/pulsar/broker_stats.go +++ b/pulsaradmin/pkg/pulsar/broker_stats.go @@ -17,22 +17,26 @@ package pulsar +import ( + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" +) + // BrokerStats is admin interface for broker stats management type BrokerStats interface { // GetMetrics returns Monitoring metrics - GetMetrics() ([]Metrics, error) + GetMetrics() ([]utils.Metrics, error) // GetMBeans requests JSON string server mbean dump - GetMBeans() ([]Metrics, error) + GetMBeans() ([]utils.Metrics, error) // GetTopics returns JSON string topics stats GetTopics() (string, error) // GetLoadReport returns load report of broker - GetLoadReport() (*LocalBrokerData, error) + GetLoadReport() (*utils.LocalBrokerData, error) // GetAllocatorStats returns stats from broker - GetAllocatorStats(allocatorName string) (*AllocatorStats, error) + GetAllocatorStats(allocatorName string) (*utils.AllocatorStats, error) } type brokerStats struct { @@ -48,9 +52,9 @@ func (c *client) BrokerStats() BrokerStats { } } -func (bs *brokerStats) GetMetrics() ([]Metrics, error) { +func (bs *brokerStats) GetMetrics() ([]utils.Metrics, error) { endpoint := bs.client.endpoint(bs.basePath, "/metrics") - var response []Metrics + var response []utils.Metrics err := bs.client.get(endpoint, &response) if err != nil { return nil, err @@ -59,9 +63,9 @@ func (bs *brokerStats) GetMetrics() ([]Metrics, error) { return response, nil } -func (bs *brokerStats) GetMBeans() ([]Metrics, error) { +func (bs *brokerStats) GetMBeans() ([]utils.Metrics, error) { endpoint := bs.client.endpoint(bs.basePath, "/mbeans") - var response []Metrics + var response []utils.Metrics err := bs.client.get(endpoint, &response) if err != nil { return nil, err @@ -80,9 +84,9 @@ func (bs *brokerStats) GetTopics() (string, error) { return string(buf), nil } -func (bs *brokerStats) GetLoadReport() (*LocalBrokerData, error) { +func (bs *brokerStats) GetLoadReport() (*utils.LocalBrokerData, error) { endpoint := bs.client.endpoint(bs.basePath, "/load-report") - response := NewLocalBrokerData() + response := utils.NewLocalBrokerData() err := bs.client.get(endpoint, &response) if err != nil { return nil, nil @@ -90,9 +94,9 @@ func (bs *brokerStats) GetLoadReport() (*LocalBrokerData, error) { return &response, nil } -func (bs *brokerStats) GetAllocatorStats(allocatorName string) (*AllocatorStats, error) { +func (bs *brokerStats) GetAllocatorStats(allocatorName string) (*utils.AllocatorStats, error) { endpoint := bs.client.endpoint(bs.basePath, "/allocator-stats", allocatorName) - var allocatorStats AllocatorStats + var allocatorStats utils.AllocatorStats err := bs.client.get(endpoint, &allocatorStats) if err != nil { return nil, err diff --git a/pulsaradmin/pkg/pulsar/brokers.go b/pulsaradmin/pkg/pulsar/brokers.go index 1b79af0315..a292397a98 100644 --- a/pulsaradmin/pkg/pulsar/brokers.go +++ b/pulsaradmin/pkg/pulsar/brokers.go @@ -21,6 +21,8 @@ import ( "fmt" "net/url" "strings" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) // Brokers is admin interface for brokers management @@ -32,7 +34,7 @@ type Brokers interface { GetDynamicConfigurationNames() ([]string, error) // GetOwnedNamespaces returns the map of owned namespaces and their status from a single broker in the cluster - GetOwnedNamespaces(cluster, brokerURL string) (map[string]NamespaceOwnershipStatus, error) + GetOwnedNamespaces(cluster, brokerURL string) (map[string]utils.NamespaceOwnershipStatus, error) // UpdateDynamicConfiguration updates dynamic configuration value in to Zk that triggers watch on // brokers and all brokers can update {@link ServiceConfiguration} value locally @@ -46,7 +48,7 @@ type Brokers interface { GetRuntimeConfigurations() (map[string]string, error) // GetInternalConfigurationData returns the internal configuration data - GetInternalConfigurationData() (*InternalConfigurationData, error) + GetInternalConfigurationData() (*utils.InternalConfigurationData, error) // GetAllDynamicConfigurations returns values of all overridden dynamic-configs GetAllDynamicConfigurations() (map[string]string, error) @@ -88,9 +90,9 @@ func (b *broker) GetDynamicConfigurationNames() ([]string, error) { return res, nil } -func (b *broker) GetOwnedNamespaces(cluster, brokerURL string) (map[string]NamespaceOwnershipStatus, error) { +func (b *broker) GetOwnedNamespaces(cluster, brokerURL string) (map[string]utils.NamespaceOwnershipStatus, error) { endpoint := b.client.endpoint(b.basePath, cluster, brokerURL, "ownedNamespaces") - var res map[string]NamespaceOwnershipStatus + var res map[string]utils.NamespaceOwnershipStatus err := b.client.get(endpoint, &res) if err != nil { return nil, err @@ -119,9 +121,9 @@ func (b *broker) GetRuntimeConfigurations() (map[string]string, error) { return res, nil } -func (b *broker) GetInternalConfigurationData() (*InternalConfigurationData, error) { +func (b *broker) GetInternalConfigurationData() (*utils.InternalConfigurationData, error) { endpoint := b.client.endpoint(b.basePath, "/internal-configuration") - var res InternalConfigurationData + var res utils.InternalConfigurationData err := b.client.get(endpoint, &res) if err != nil { return nil, err diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index 08ea892819..3f9cc4309b 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -17,22 +17,26 @@ package pulsar +import ( + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" +) + // Clusters is admin interface for clusters management type Clusters interface { // List returns the list of clusters List() ([]string, error) // Get the configuration data for the specified cluster - Get(string) (ClusterData, error) + Get(string) (utils.ClusterData, error) // Create a new cluster - Create(ClusterData) error + Create(utils.ClusterData) error // Delete an existing cluster Delete(string) error // Update the configuration for a cluster - Update(ClusterData) error + Update(utils.ClusterData) error // UpdatePeerClusters updates peer cluster names. UpdatePeerClusters(string, []string) error @@ -41,19 +45,19 @@ type Clusters interface { GetPeerClusters(string) ([]string, error) // CreateFailureDomain creates a domain into cluster - CreateFailureDomain(FailureDomainData) error + CreateFailureDomain(utils.FailureDomainData) error // GetFailureDomain returns the domain registered into a cluster - GetFailureDomain(clusterName, domainName string) (FailureDomainData, error) + GetFailureDomain(clusterName, domainName string) (utils.FailureDomainData, error) // ListFailureDomains returns all registered domains in cluster - ListFailureDomains(string) (FailureDomainMap, error) + ListFailureDomains(string) (utils.FailureDomainMap, error) // DeleteFailureDomain deletes a domain in cluster - DeleteFailureDomain(FailureDomainData) error + DeleteFailureDomain(utils.FailureDomainData) error // UpdateFailureDomain updates a domain into cluster - UpdateFailureDomain(FailureDomainData) error + UpdateFailureDomain(utils.FailureDomainData) error } type clusters struct { @@ -75,14 +79,14 @@ func (c *clusters) List() ([]string, error) { return clusters, err } -func (c *clusters) Get(name string) (ClusterData, error) { - cdata := ClusterData{} +func (c *clusters) Get(name string) (utils.ClusterData, error) { + cdata := utils.ClusterData{} endpoint := c.client.endpoint(c.basePath, name) err := c.client.get(endpoint, &cdata) return cdata, err } -func (c *clusters) Create(cdata ClusterData) error { +func (c *clusters) Create(cdata utils.ClusterData) error { endpoint := c.client.endpoint(c.basePath, cdata.Name) return c.client.put(endpoint, &cdata) } @@ -92,7 +96,7 @@ func (c *clusters) Delete(name string) error { return c.client.delete(endpoint) } -func (c *clusters) Update(cdata ClusterData) error { +func (c *clusters) Update(cdata utils.ClusterData) error { endpoint := c.client.endpoint(c.basePath, cdata.Name) return c.client.post(endpoint, &cdata) } @@ -109,30 +113,30 @@ func (c *clusters) UpdatePeerClusters(cluster string, peerClusters []string) err return c.client.post(endpoint, peerClusters) } -func (c *clusters) CreateFailureDomain(data FailureDomainData) error { +func (c *clusters) CreateFailureDomain(data utils.FailureDomainData) error { endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) return c.client.post(endpoint, &data) } -func (c *clusters) GetFailureDomain(clusterName string, domainName string) (FailureDomainData, error) { - var res FailureDomainData +func (c *clusters) GetFailureDomain(clusterName string, domainName string) (utils.FailureDomainData, error) { + var res utils.FailureDomainData endpoint := c.client.endpoint(c.basePath, clusterName, "failureDomains", domainName) err := c.client.get(endpoint, &res) return res, err } -func (c *clusters) ListFailureDomains(clusterName string) (FailureDomainMap, error) { - var domainData FailureDomainMap +func (c *clusters) ListFailureDomains(clusterName string) (utils.FailureDomainMap, error) { + var domainData utils.FailureDomainMap endpoint := c.client.endpoint(c.basePath, clusterName, "failureDomains") err := c.client.get(endpoint, &domainData) return domainData, err } -func (c *clusters) DeleteFailureDomain(data FailureDomainData) error { +func (c *clusters) DeleteFailureDomain(data utils.FailureDomainData) error { endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) return c.client.delete(endpoint) } -func (c *clusters) UpdateFailureDomain(data FailureDomainData) error { +func (c *clusters) UpdateFailureDomain(data utils.FailureDomainData) error { endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) return c.client.post(endpoint, &data) } diff --git a/pulsaradmin/pkg/pulsar/api_version.go b/pulsaradmin/pkg/pulsar/common/api_version.go similarity index 98% rename from pulsaradmin/pkg/pulsar/api_version.go rename to pulsaradmin/pkg/pulsar/common/api_version.go index 70963d9434..f490828daf 100644 --- a/pulsaradmin/pkg/pulsar/api_version.go +++ b/pulsaradmin/pkg/pulsar/common/api_version.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package common type APIVersion int diff --git a/pulsaradmin/pkg/pulsar/api_version_test.go b/pulsaradmin/pkg/pulsar/common/api_version_test.go similarity index 98% rename from pulsaradmin/pkg/pulsar/api_version_test.go rename to pulsaradmin/pkg/pulsar/common/api_version_test.go index adcddccfa3..5281922dfb 100644 --- a/pulsaradmin/pkg/pulsar/api_version_test.go +++ b/pulsaradmin/pkg/pulsar/common/api_version_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package common import ( "testing" diff --git a/pulsaradmin/pkg/pulsar/auth_action.go b/pulsaradmin/pkg/pulsar/common/auth_action.go similarity index 98% rename from pulsaradmin/pkg/pulsar/auth_action.go rename to pulsaradmin/pkg/pulsar/common/auth_action.go index 84e42debbb..2983c94f64 100644 --- a/pulsaradmin/pkg/pulsar/auth_action.go +++ b/pulsaradmin/pkg/pulsar/common/auth_action.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package common import "github.com/pkg/errors" diff --git a/pulsaradmin/pkg/pulsar/auth_polices.go b/pulsaradmin/pkg/pulsar/common/auth_polices.go similarity index 98% rename from pulsaradmin/pkg/pulsar/auth_polices.go rename to pulsaradmin/pkg/pulsar/common/auth_polices.go index 96a906cdbb..dd6dee82bf 100644 --- a/pulsaradmin/pkg/pulsar/auth_polices.go +++ b/pulsaradmin/pkg/pulsar/common/auth_polices.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package common type AuthPolicies struct { NamespaceAuth map[string]AuthAction `json:"namespace_auth"` diff --git a/pulsaradmin/pkg/pulsar/descriptions.go b/pulsaradmin/pkg/pulsar/common/descriptions.go similarity index 99% rename from pulsaradmin/pkg/pulsar/descriptions.go rename to pulsaradmin/pkg/pulsar/common/descriptions.go index 513e73f7d7..59856db5f0 100644 --- a/pulsaradmin/pkg/pulsar/descriptions.go +++ b/pulsaradmin/pkg/pulsar/common/descriptions.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package common import ( "strings" diff --git a/pulsaradmin/pkg/pulsar/descriptions_test.go b/pulsaradmin/pkg/pulsar/common/descriptions_test.go similarity index 99% rename from pulsaradmin/pkg/pulsar/descriptions_test.go rename to pulsaradmin/pkg/pulsar/common/descriptions_test.go index 361fdd5f2c..661bb41f88 100644 --- a/pulsaradmin/pkg/pulsar/descriptions_test.go +++ b/pulsaradmin/pkg/pulsar/common/descriptions_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package common import ( "testing" diff --git a/pulsaradmin/pkg/pulsar/errors.go b/pulsaradmin/pkg/pulsar/common/errors.go similarity index 94% rename from pulsaradmin/pkg/pulsar/errors.go rename to pulsaradmin/pkg/pulsar/common/errors.go index 10ba9503d9..191d936630 100644 --- a/pulsaradmin/pkg/pulsar/errors.go +++ b/pulsaradmin/pkg/pulsar/common/errors.go @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package common import "fmt" -const unknownErrorReason = "Unknown pulsar error" +const UnknownErrorReason = "Unknown pulsar error" type Error struct { Reason string `json:"reason"` diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index 127fe79ff8..6f5c751e36 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -27,12 +27,14 @@ import ( "os" "path/filepath" "strings" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) // Functions is admin interface for functions management type Functions interface { // CreateFunc create a new function. - CreateFunc(data *FunctionConfig, fileName string) error + CreateFunc(data *utils.FunctionConfig, fileName string) error // CreateFuncWithURL create a new function by providing url from which fun-pkg can be downloaded. // supported url: http/file @@ -44,7 +46,7 @@ type Functions interface { // the function configuration object // @param pkgURL // url from which pkg can be downloaded - CreateFuncWithURL(data *FunctionConfig, pkgURL string) error + CreateFuncWithURL(data *utils.FunctionConfig, pkgURL string) error // StopFunction stop all function instances StopFunction(tenant, namespace, name string) error @@ -71,34 +73,35 @@ type Functions interface { GetFunctions(tenant, namespace string) ([]string, error) // GetFunction returns the configuration for the specified function - GetFunction(tenant, namespace, name string) (FunctionConfig, error) + GetFunction(tenant, namespace, name string) (utils.FunctionConfig, error) // GetFunctionStatus returns the current status of a function - GetFunctionStatus(tenant, namespace, name string) (FunctionStatus, error) + GetFunctionStatus(tenant, namespace, name string) (utils.FunctionStatus, error) // GetFunctionStatusWithInstanceID returns the current status of a function instance - GetFunctionStatusWithInstanceID(tenant, namespace, name string, instanceID int) (FunctionInstanceStatusData, error) + GetFunctionStatusWithInstanceID(tenant, namespace, name string, instanceID int) ( + utils.FunctionInstanceStatusData, error) // GetFunctionStats returns the current stats of a function - GetFunctionStats(tenant, namespace, name string) (FunctionStats, error) + GetFunctionStats(tenant, namespace, name string) (utils.FunctionStats, error) // GetFunctionStatsWithInstanceID gets the current stats of a function instance - GetFunctionStatsWithInstanceID(tenant, namespace, name string, instanceID int) (FunctionInstanceStatsData, error) + GetFunctionStatsWithInstanceID(tenant, namespace, name string, instanceID int) (utils.FunctionInstanceStatsData, error) // GetFunctionState fetch the current state associated with a Pulsar Function // // Response Example: // { "value : 12, version : 2"} - GetFunctionState(tenant, namespace, name, key string) (FunctionState, error) + GetFunctionState(tenant, namespace, name, key string) (utils.FunctionState, error) // PutFunctionState puts the given state associated with a Pulsar Function - PutFunctionState(tenant, namespace, name string, state FunctionState) error + PutFunctionState(tenant, namespace, name string, state utils.FunctionState) error // TriggerFunction triggers the function by writing to the input topic TriggerFunction(tenant, namespace, name, topic, triggerValue, triggerFile string) (string, error) // UpdateFunction updates the configuration for a function. - UpdateFunction(functionConfig *FunctionConfig, fileName string, updateOptions *UpdateOptions) error + UpdateFunction(functionConfig *utils.FunctionConfig, fileName string, updateOptions *utils.UpdateOptions) error // UpdateFunctionWithURL updates the configuration for a function. // @@ -106,7 +109,7 @@ type Functions interface { // eg: // File: file:/dir/fileName.jar // Http: http://www.repo.com/fileName.jar - UpdateFunctionWithURL(functionConfig *FunctionConfig, pkgURL string, updateOptions *UpdateOptions) error + UpdateFunctionWithURL(functionConfig *utils.FunctionConfig, pkgURL string, updateOptions *utils.UpdateOptions) error } type functions struct { @@ -136,7 +139,7 @@ func (f *functions) createTextFromFiled(w *multipart.Writer, value string) (io.W return w.CreatePart(h) } -func (f *functions) CreateFunc(funcConf *FunctionConfig, fileName string) error { +func (f *functions) CreateFunc(funcConf *utils.FunctionConfig, fileName string) error { endpoint := f.client.endpoint(f.basePath, funcConf.Tenant, funcConf.Namespace, funcConf.Name) // buffer to store our request as bytes @@ -195,7 +198,7 @@ func (f *functions) CreateFunc(funcConf *FunctionConfig, fileName string) error return nil } -func (f *functions) CreateFuncWithURL(funcConf *FunctionConfig, pkgURL string) error { +func (f *functions) CreateFuncWithURL(funcConf *utils.FunctionConfig, pkgURL string) error { endpoint := f.client.endpoint(f.basePath, funcConf.Tenant, funcConf.Namespace, funcConf.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -288,15 +291,15 @@ func (f *functions) GetFunctions(tenant, namespace string) ([]string, error) { return functions, err } -func (f *functions) GetFunction(tenant, namespace, name string) (FunctionConfig, error) { - var functionConfig FunctionConfig +func (f *functions) GetFunction(tenant, namespace, name string) (utils.FunctionConfig, error) { + var functionConfig utils.FunctionConfig endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) err := f.client.get(endpoint, &functionConfig) return functionConfig, err } -func (f *functions) UpdateFunction(functionConfig *FunctionConfig, fileName string, - updateOptions *UpdateOptions) error { +func (f *functions) UpdateFunction(functionConfig *utils.FunctionConfig, fileName string, + updateOptions *utils.UpdateOptions) error { endpoint := f.client.endpoint(f.basePath, functionConfig.Tenant, functionConfig.Namespace, functionConfig.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -371,8 +374,8 @@ func (f *functions) UpdateFunction(functionConfig *FunctionConfig, fileName stri return nil } -func (f *functions) UpdateFunctionWithURL(functionConfig *FunctionConfig, pkgURL string, - updateOptions *UpdateOptions) error { +func (f *functions) UpdateFunctionWithURL(functionConfig *utils.FunctionConfig, pkgURL string, + updateOptions *utils.UpdateOptions) error { endpoint := f.client.endpoint(f.basePath, functionConfig.Tenant, functionConfig.Namespace, functionConfig.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -436,46 +439,46 @@ func (f *functions) UpdateFunctionWithURL(functionConfig *FunctionConfig, pkgURL return nil } -func (f *functions) GetFunctionStatus(tenant, namespace, name string) (FunctionStatus, error) { - var functionStatus FunctionStatus +func (f *functions) GetFunctionStatus(tenant, namespace, name string) (utils.FunctionStatus, error) { + var functionStatus utils.FunctionStatus endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) err := f.client.get(endpoint+"/status", &functionStatus) return functionStatus, err } func (f *functions) GetFunctionStatusWithInstanceID(tenant, namespace, name string, - instanceID int) (FunctionInstanceStatusData, error) { - var functionInstanceStatusData FunctionInstanceStatusData + instanceID int) (utils.FunctionInstanceStatusData, error) { + var functionInstanceStatusData utils.FunctionInstanceStatusData id := fmt.Sprintf("%d", instanceID) endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) err := f.client.get(endpoint+"/status", &functionInstanceStatusData) return functionInstanceStatusData, err } -func (f *functions) GetFunctionStats(tenant, namespace, name string) (FunctionStats, error) { - var functionStats FunctionStats +func (f *functions) GetFunctionStats(tenant, namespace, name string) (utils.FunctionStats, error) { + var functionStats utils.FunctionStats endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) err := f.client.get(endpoint+"/stats", &functionStats) return functionStats, err } func (f *functions) GetFunctionStatsWithInstanceID(tenant, namespace, name string, - instanceID int) (FunctionInstanceStatsData, error) { - var functionInstanceStatsData FunctionInstanceStatsData + instanceID int) (utils.FunctionInstanceStatsData, error) { + var functionInstanceStatsData utils.FunctionInstanceStatsData id := fmt.Sprintf("%d", instanceID) endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) err := f.client.get(endpoint+"/stats", &functionInstanceStatsData) return functionInstanceStatsData, err } -func (f *functions) GetFunctionState(tenant, namespace, name, key string) (FunctionState, error) { - var functionState FunctionState +func (f *functions) GetFunctionState(tenant, namespace, name, key string) (utils.FunctionState, error) { + var functionState utils.FunctionState endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, "state", key) err := f.client.get(endpoint, &functionState) return functionState, err } -func (f *functions) PutFunctionState(tenant, namespace, name string, state FunctionState) error { +func (f *functions) PutFunctionState(tenant, namespace, name string, state utils.FunctionState) error { endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, "state", state.Key) // buffer to store our request as bytes diff --git a/pulsaradmin/pkg/pulsar/functions_worker.go b/pulsaradmin/pkg/pulsar/functions_worker.go index acdaf85c31..8ce8a5eb3e 100644 --- a/pulsaradmin/pkg/pulsar/functions_worker.go +++ b/pulsaradmin/pkg/pulsar/functions_worker.go @@ -17,18 +17,22 @@ package pulsar +import ( + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" +) + type FunctionsWorker interface { // Get all functions stats on a worker - GetFunctionsStats() ([]*WorkerFunctionInstanceStats, error) + GetFunctionsStats() ([]*utils.WorkerFunctionInstanceStats, error) // Get worker metrics - GetMetrics() ([]*Metrics, error) + GetMetrics() ([]*utils.Metrics, error) // Get List of all workers belonging to this cluster - GetCluster() ([]*WorkerInfo, error) + GetCluster() ([]*utils.WorkerInfo, error) // Get the worker who is the leader of the clusterv - GetClusterLeader() (*WorkerInfo, error) + GetClusterLeader() (*utils.WorkerInfo, error) // Get the function assignment among the cluster GetAssignments() (map[string][]string, error) @@ -48,9 +52,9 @@ func (c *client) FunctionsWorker() FunctionsWorker { } } -func (w *worker) GetFunctionsStats() ([]*WorkerFunctionInstanceStats, error) { +func (w *worker) GetFunctionsStats() ([]*utils.WorkerFunctionInstanceStats, error) { endpoint := w.client.endpoint(w.workerStatsPath, "functionsmetrics") - var workerStats []*WorkerFunctionInstanceStats + var workerStats []*utils.WorkerFunctionInstanceStats err := w.client.get(endpoint, &workerStats) if err != nil { return nil, err @@ -58,9 +62,9 @@ func (w *worker) GetFunctionsStats() ([]*WorkerFunctionInstanceStats, error) { return workerStats, nil } -func (w *worker) GetMetrics() ([]*Metrics, error) { +func (w *worker) GetMetrics() ([]*utils.Metrics, error) { endpoint := w.client.endpoint(w.workerStatsPath, "metrics") - var metrics []*Metrics + var metrics []*utils.Metrics err := w.client.get(endpoint, &metrics) if err != nil { return nil, err @@ -68,9 +72,9 @@ func (w *worker) GetMetrics() ([]*Metrics, error) { return metrics, nil } -func (w *worker) GetCluster() ([]*WorkerInfo, error) { +func (w *worker) GetCluster() ([]*utils.WorkerInfo, error) { endpoint := w.client.endpoint(w.workerPath, "cluster") - var workersInfo []*WorkerInfo + var workersInfo []*utils.WorkerInfo err := w.client.get(endpoint, &workersInfo) if err != nil { return nil, err @@ -78,9 +82,9 @@ func (w *worker) GetCluster() ([]*WorkerInfo, error) { return workersInfo, nil } -func (w *worker) GetClusterLeader() (*WorkerInfo, error) { +func (w *worker) GetClusterLeader() (*utils.WorkerInfo, error) { endpoint := w.client.endpoint(w.workerPath, "cluster", "leader") - var workerInfo WorkerInfo + var workerInfo utils.WorkerInfo err := w.client.get(endpoint, &workerInfo) if err != nil { return nil, err diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index 43375a379b..02a91a9c5b 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -21,6 +21,9 @@ import ( "net/url" "strconv" "strings" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) // Namespaces is admin interface for namespaces management @@ -32,7 +35,7 @@ type Namespaces interface { GetTopics(namespace string) ([]string, error) // GetPolicies returns the dump all the policies specified for a namespace - GetPolicies(namespace string) (*Policies, error) + GetPolicies(namespace string) (*utils.Policies, error) // CreateNamespace creates a new empty namespace with no policies attached CreateNamespace(namespace string) error @@ -41,10 +44,10 @@ type Namespaces interface { CreateNsWithNumBundles(namespace string, numBundles int) error // CreateNsWithPolices creates a new namespace with the specified policies - CreateNsWithPolices(namespace string, polices Policies) error + CreateNsWithPolices(namespace string, polices utils.Policies) error // CreateNsWithBundlesData creates a new empty namespace with no policies attached - CreateNsWithBundlesData(namespace string, bundleData *BundlesData) error + CreateNsWithBundlesData(namespace string, bundleData *utils.BundlesData) error // DeleteNamespace deletes an existing namespace DeleteNamespace(namespace string) error @@ -59,72 +62,73 @@ type Namespaces interface { GetNamespaceMessageTTL(namespace string) (int, error) // GetRetention returns the retention configuration for a namespace - GetRetention(namespace string) (*RetentionPolicies, error) + GetRetention(namespace string) (*utils.RetentionPolicies, error) // SetRetention sets the retention configuration for all the topics on a namespace - SetRetention(namespace string, policy RetentionPolicies) error + SetRetention(namespace string, policy utils.RetentionPolicies) error // GetBacklogQuotaMap returns backlog quota map on a namespace - GetBacklogQuotaMap(namespace string) (map[BacklogQuotaType]BacklogQuota, error) + GetBacklogQuotaMap(namespace string) (map[utils.BacklogQuotaType]utils.BacklogQuota, error) // SetBacklogQuota sets a backlog quota for all the topics on a namespace - SetBacklogQuota(namespace string, backlogQuota BacklogQuota) error + SetBacklogQuota(namespace string, backlogQuota utils.BacklogQuota) error // RemoveBacklogQuota removes a backlog quota policy from a namespace RemoveBacklogQuota(namespace string) error // SetSchemaValidationEnforced sets schema validation enforced for namespace - SetSchemaValidationEnforced(namespace NameSpaceName, schemaValidationEnforced bool) error + SetSchemaValidationEnforced(namespace utils.NameSpaceName, schemaValidationEnforced bool) error // GetSchemaValidationEnforced returns schema validation enforced for namespace - GetSchemaValidationEnforced(namespace NameSpaceName) (bool, error) + GetSchemaValidationEnforced(namespace utils.NameSpaceName) (bool, error) // SetSchemaAutoUpdateCompatibilityStrategy sets the strategy used to check the a new schema provided // by a producer is compatible with the current schema before it is installed - SetSchemaAutoUpdateCompatibilityStrategy(namespace NameSpaceName, strategy SchemaCompatibilityStrategy) error + SetSchemaAutoUpdateCompatibilityStrategy(namespace utils.NameSpaceName, + strategy utils.SchemaCompatibilityStrategy) error // GetSchemaAutoUpdateCompatibilityStrategy returns the strategy used to check the a new schema provided // by a producer is compatible with the current schema before it is installed - GetSchemaAutoUpdateCompatibilityStrategy(namespace NameSpaceName) (SchemaCompatibilityStrategy, error) + GetSchemaAutoUpdateCompatibilityStrategy(namespace utils.NameSpaceName) (utils.SchemaCompatibilityStrategy, error) // ClearOffloadDeleteLag clears the offload deletion lag for a namespace. - ClearOffloadDeleteLag(namespace NameSpaceName) error + ClearOffloadDeleteLag(namespace utils.NameSpaceName) error // SetOffloadDeleteLag sets the offload deletion lag for a namespace - SetOffloadDeleteLag(namespace NameSpaceName, timeMs int64) error + SetOffloadDeleteLag(namespace utils.NameSpaceName, timeMs int64) error // GetOffloadDeleteLag returns the offload deletion lag for a namespace, in milliseconds - GetOffloadDeleteLag(namespace NameSpaceName) (int64, error) + GetOffloadDeleteLag(namespace utils.NameSpaceName) (int64, error) // SetOffloadThreshold sets the offloadThreshold for a namespace - SetOffloadThreshold(namespace NameSpaceName, threshold int64) error + SetOffloadThreshold(namespace utils.NameSpaceName, threshold int64) error // GetOffloadThreshold returns the offloadThreshold for a namespace - GetOffloadThreshold(namespace NameSpaceName) (int64, error) + GetOffloadThreshold(namespace utils.NameSpaceName) (int64, error) // SetCompactionThreshold sets the compactionThreshold for a namespace - SetCompactionThreshold(namespace NameSpaceName, threshold int64) error + SetCompactionThreshold(namespace utils.NameSpaceName, threshold int64) error // GetCompactionThreshold returns the compactionThreshold for a namespace - GetCompactionThreshold(namespace NameSpaceName) (int64, error) + GetCompactionThreshold(namespace utils.NameSpaceName) (int64, error) // SetMaxConsumersPerSubscription sets maxConsumersPerSubscription for a namespace. - SetMaxConsumersPerSubscription(namespace NameSpaceName, max int) error + SetMaxConsumersPerSubscription(namespace utils.NameSpaceName, max int) error // GetMaxConsumersPerSubscription returns the maxConsumersPerSubscription for a namespace. - GetMaxConsumersPerSubscription(namespace NameSpaceName) (int, error) + GetMaxConsumersPerSubscription(namespace utils.NameSpaceName) (int, error) // SetMaxConsumersPerTopic sets maxConsumersPerTopic for a namespace. - SetMaxConsumersPerTopic(namespace NameSpaceName, max int) error + SetMaxConsumersPerTopic(namespace utils.NameSpaceName, max int) error // GetMaxConsumersPerTopic returns the maxProducersPerTopic for a namespace. - GetMaxConsumersPerTopic(namespace NameSpaceName) (int, error) + GetMaxConsumersPerTopic(namespace utils.NameSpaceName) (int, error) // SetMaxProducersPerTopic sets maxProducersPerTopic for a namespace. - SetMaxProducersPerTopic(namespace NameSpaceName, max int) error + SetMaxProducersPerTopic(namespace utils.NameSpaceName, max int) error // GetMaxProducersPerTopic returns the maxProducersPerTopic for a namespace. - GetMaxProducersPerTopic(namespace NameSpaceName) (int, error) + GetMaxProducersPerTopic(namespace utils.NameSpaceName) (int, error) // GetNamespaceReplicationClusters returns the replication clusters for a namespace GetNamespaceReplicationClusters(namespace string) ([]string, error) @@ -149,20 +153,20 @@ type Namespaces interface { SetDeduplicationStatus(namespace string, enableDeduplication bool) error // SetPersistence sets the persistence configuration for all the topics on a namespace - SetPersistence(namespace string, persistence PersistencePolicies) error + SetPersistence(namespace string, persistence utils.PersistencePolicies) error // GetPersistence returns the persistence configuration for a namespace - GetPersistence(namespace string) (*PersistencePolicies, error) + GetPersistence(namespace string) (*utils.PersistencePolicies, error) // SetBookieAffinityGroup sets bookie affinity group for a namespace to isolate namespace write to bookies that are // part of given affinity group - SetBookieAffinityGroup(namespace string, bookieAffinityGroup BookieAffinityGroupData) error + SetBookieAffinityGroup(namespace string, bookieAffinityGroup utils.BookieAffinityGroupData) error // DeleteBookieAffinityGroup deletes bookie affinity group configured for a namespace DeleteBookieAffinityGroup(namespace string) error // GetBookieAffinityGroup returns bookie affinity group configured for a namespace - GetBookieAffinityGroup(namespace string) (*BookieAffinityGroupData, error) + GetBookieAffinityGroup(namespace string) (*utils.BookieAffinityGroupData, error) // Unload a namespace from the current serving broker Unload(namespace string) error @@ -174,75 +178,75 @@ type Namespaces interface { SplitNamespaceBundle(namespace, bundle string, unloadSplitBundles bool) error // GetNamespacePermissions returns permissions on a namespace - GetNamespacePermissions(namespace NameSpaceName) (map[string][]AuthAction, error) + GetNamespacePermissions(namespace utils.NameSpaceName) (map[string][]common.AuthAction, error) // GrantNamespacePermission grants permission on a namespace. - GrantNamespacePermission(namespace NameSpaceName, role string, action []AuthAction) error + GrantNamespacePermission(namespace utils.NameSpaceName, role string, action []common.AuthAction) error // RevokeNamespacePermission revokes permissions on a namespace. - RevokeNamespacePermission(namespace NameSpaceName, role string) error + RevokeNamespacePermission(namespace utils.NameSpaceName, role string) error // GrantSubPermission grants permission to role to access subscription's admin-api - GrantSubPermission(namespace NameSpaceName, sName string, roles []string) error + GrantSubPermission(namespace utils.NameSpaceName, sName string, roles []string) error // RevokeSubPermission revoke permissions on a subscription's admin-api access - RevokeSubPermission(namespace NameSpaceName, sName, role string) error + RevokeSubPermission(namespace utils.NameSpaceName, sName, role string) error // SetSubscriptionAuthMode sets the given subscription auth mode on all topics on a namespace - SetSubscriptionAuthMode(namespace NameSpaceName, mode SubscriptionAuthMode) error + SetSubscriptionAuthMode(namespace utils.NameSpaceName, mode utils.SubscriptionAuthMode) error // SetEncryptionRequiredStatus sets the encryption required status for all topics within a namespace - SetEncryptionRequiredStatus(namespace NameSpaceName, encrypt bool) error + SetEncryptionRequiredStatus(namespace utils.NameSpaceName, encrypt bool) error // UnsubscribeNamespace unsubscribe the given subscription on all topics on a namespace - UnsubscribeNamespace(namespace NameSpaceName, sName string) error + UnsubscribeNamespace(namespace utils.NameSpaceName, sName string) error // UnsubscribeNamespaceBundle unsubscribe the given subscription on all topics on a namespace bundle - UnsubscribeNamespaceBundle(namespace NameSpaceName, bundle, sName string) error + UnsubscribeNamespaceBundle(namespace utils.NameSpaceName, bundle, sName string) error // ClearNamespaceBundleBacklogForSubscription clears backlog for a given subscription on all // topics on a namespace bundle - ClearNamespaceBundleBacklogForSubscription(namespace NameSpaceName, bundle, sName string) error + ClearNamespaceBundleBacklogForSubscription(namespace utils.NameSpaceName, bundle, sName string) error // ClearNamespaceBundleBacklog clears backlog for all topics on a namespace bundle - ClearNamespaceBundleBacklog(namespace NameSpaceName, bundle string) error + ClearNamespaceBundleBacklog(namespace utils.NameSpaceName, bundle string) error // ClearNamespaceBacklogForSubscription clears backlog for a given subscription on all topics on a namespace - ClearNamespaceBacklogForSubscription(namespace NameSpaceName, sName string) error + ClearNamespaceBacklogForSubscription(namespace utils.NameSpaceName, sName string) error // ClearNamespaceBacklog clears backlog for all topics on a namespace - ClearNamespaceBacklog(namespace NameSpaceName) error + ClearNamespaceBacklog(namespace utils.NameSpaceName) error // SetReplicatorDispatchRate sets replicator-Message-dispatch-rate (Replicators under this namespace // can dispatch this many messages per second) - SetReplicatorDispatchRate(namespace NameSpaceName, rate DispatchRate) error + SetReplicatorDispatchRate(namespace utils.NameSpaceName, rate utils.DispatchRate) error // Get replicator-Message-dispatch-rate (Replicators under this namespace // can dispatch this many messages per second) - GetReplicatorDispatchRate(namespace NameSpaceName) (DispatchRate, error) + GetReplicatorDispatchRate(namespace utils.NameSpaceName) (utils.DispatchRate, error) // SetSubscriptionDispatchRate sets subscription-Message-dispatch-rate (subscriptions under this namespace // can dispatch this many messages per second) - SetSubscriptionDispatchRate(namespace NameSpaceName, rate DispatchRate) error + SetSubscriptionDispatchRate(namespace utils.NameSpaceName, rate utils.DispatchRate) error // GetSubscriptionDispatchRate returns subscription-Message-dispatch-rate (subscriptions under this namespace // can dispatch this many messages per second) - GetSubscriptionDispatchRate(namespace NameSpaceName) (DispatchRate, error) + GetSubscriptionDispatchRate(namespace utils.NameSpaceName) (utils.DispatchRate, error) // SetSubscribeRate sets namespace-subscribe-rate (topics under this namespace will limit by subscribeRate) - SetSubscribeRate(namespace NameSpaceName, rate SubscribeRate) error + SetSubscribeRate(namespace utils.NameSpaceName, rate utils.SubscribeRate) error // GetSubscribeRate returns namespace-subscribe-rate (topics under this namespace allow subscribe // times per consumer in a period) - GetSubscribeRate(namespace NameSpaceName) (SubscribeRate, error) + GetSubscribeRate(namespace utils.NameSpaceName) (utils.SubscribeRate, error) // SetDispatchRate sets Message-dispatch-rate (topics under this namespace can dispatch // this many messages per second) - SetDispatchRate(namespace NameSpaceName, rate DispatchRate) error + SetDispatchRate(namespace utils.NameSpaceName, rate utils.DispatchRate) error // GetDispatchRate returns Message-dispatch-rate (topics under this namespace can dispatch // this many messages per second) - GetDispatchRate(namespace NameSpaceName) (DispatchRate, error) + GetDispatchRate(namespace utils.NameSpaceName) (utils.DispatchRate, error) } type namespaces struct { @@ -267,7 +271,7 @@ func (n *namespaces) GetNamespaces(tenant string) ([]string, error) { func (n *namespaces) GetTopics(namespace string) ([]string, error) { var topics []string - ns, err := GetNamespaceName(namespace) + ns, err := utils.GetNamespaceName(namespace) if err != nil { return nil, err } @@ -276,9 +280,9 @@ func (n *namespaces) GetTopics(namespace string) ([]string, error) { return topics, err } -func (n *namespaces) GetPolicies(namespace string) (*Policies, error) { - var police Policies - ns, err := GetNamespaceName(namespace) +func (n *namespaces) GetPolicies(namespace string) (*utils.Policies, error) { + var police utils.Policies + ns, err := utils.GetNamespaceName(namespace) if err != nil { return nil, err } @@ -288,11 +292,11 @@ func (n *namespaces) GetPolicies(namespace string) (*Policies, error) { } func (n *namespaces) CreateNsWithNumBundles(namespace string, numBundles int) error { - return n.CreateNsWithBundlesData(namespace, NewBundlesDataWithNumBundles(numBundles)) + return n.CreateNsWithBundlesData(namespace, utils.NewBundlesDataWithNumBundles(numBundles)) } -func (n *namespaces) CreateNsWithPolices(namespace string, policies Policies) error { - ns, err := GetNamespaceName(namespace) +func (n *namespaces) CreateNsWithPolices(namespace string, policies utils.Policies) error { + ns, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -300,20 +304,20 @@ func (n *namespaces) CreateNsWithPolices(namespace string, policies Policies) er return n.client.put(endpoint, &policies) } -func (n *namespaces) CreateNsWithBundlesData(namespace string, bundleData *BundlesData) error { - ns, err := GetNamespaceName(namespace) +func (n *namespaces) CreateNsWithBundlesData(namespace string, bundleData *utils.BundlesData) error { + ns, err := utils.GetNamespaceName(namespace) if err != nil { return err } endpoint := n.client.endpoint(n.basePath, ns.String()) - polices := new(Policies) + polices := new(utils.Policies) polices.Bundles = bundleData return n.client.put(endpoint, &polices) } func (n *namespaces) CreateNamespace(namespace string) error { - ns, err := GetNamespaceName(namespace) + ns, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -322,7 +326,7 @@ func (n *namespaces) CreateNamespace(namespace string) error { } func (n *namespaces) DeleteNamespace(namespace string) error { - ns, err := GetNamespaceName(namespace) + ns, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -331,7 +335,7 @@ func (n *namespaces) DeleteNamespace(namespace string) error { } func (n *namespaces) DeleteNamespaceBundle(namespace string, bundleRange string) error { - ns, err := GetNamespaceName(namespace) + ns, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -341,7 +345,7 @@ func (n *namespaces) DeleteNamespaceBundle(namespace string, bundleRange string) func (n *namespaces) GetNamespaceMessageTTL(namespace string) (int, error) { var ttl int - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return 0, err } @@ -351,7 +355,7 @@ func (n *namespaces) GetNamespaceMessageTTL(namespace string) (int, error) { } func (n *namespaces) SetNamespaceMessageTTL(namespace string, ttlInSeconds int) error { - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -360,8 +364,8 @@ func (n *namespaces) SetNamespaceMessageTTL(namespace string, ttlInSeconds int) return n.client.post(endpoint, &ttlInSeconds) } -func (n *namespaces) SetRetention(namespace string, policy RetentionPolicies) error { - nsName, err := GetNamespaceName(namespace) +func (n *namespaces) SetRetention(namespace string, policy utils.RetentionPolicies) error { + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -369,9 +373,9 @@ func (n *namespaces) SetRetention(namespace string, policy RetentionPolicies) er return n.client.post(endpoint, &policy) } -func (n *namespaces) GetRetention(namespace string) (*RetentionPolicies, error) { - var policy RetentionPolicies - nsName, err := GetNamespaceName(namespace) +func (n *namespaces) GetRetention(namespace string) (*utils.RetentionPolicies, error) { + var policy utils.RetentionPolicies + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return nil, err } @@ -380,9 +384,9 @@ func (n *namespaces) GetRetention(namespace string) (*RetentionPolicies, error) return &policy, err } -func (n *namespaces) GetBacklogQuotaMap(namespace string) (map[BacklogQuotaType]BacklogQuota, error) { - var backlogQuotaMap map[BacklogQuotaType]BacklogQuota - nsName, err := GetNamespaceName(namespace) +func (n *namespaces) GetBacklogQuotaMap(namespace string) (map[utils.BacklogQuotaType]utils.BacklogQuota, error) { + var backlogQuotaMap map[utils.BacklogQuotaType]utils.BacklogQuota + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return nil, err } @@ -391,8 +395,8 @@ func (n *namespaces) GetBacklogQuotaMap(namespace string) (map[BacklogQuotaType] return backlogQuotaMap, err } -func (n *namespaces) SetBacklogQuota(namespace string, backlogQuota BacklogQuota) error { - nsName, err := GetNamespaceName(namespace) +func (n *namespaces) SetBacklogQuota(namespace string, backlogQuota utils.BacklogQuota) error { + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -401,23 +405,23 @@ func (n *namespaces) SetBacklogQuota(namespace string, backlogQuota BacklogQuota } func (n *namespaces) RemoveBacklogQuota(namespace string) error { - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuota") params := map[string]string{ - "backlogQuotaType": string(DestinationStorage), + "backlogQuotaType": string(utils.DestinationStorage), } return n.client.deleteWithQueryParams(endpoint, nil, params) } -func (n *namespaces) SetSchemaValidationEnforced(namespace NameSpaceName, schemaValidationEnforced bool) error { +func (n *namespaces) SetSchemaValidationEnforced(namespace utils.NameSpaceName, schemaValidationEnforced bool) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaValidationEnforced") return n.client.post(endpoint, schemaValidationEnforced) } -func (n *namespaces) GetSchemaValidationEnforced(namespace NameSpaceName) (bool, error) { +func (n *namespaces) GetSchemaValidationEnforced(namespace utils.NameSpaceName) (bool, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaValidationEnforced") r, err := n.client.getWithQueryParams(endpoint, nil, nil, false) if err != nil { @@ -426,38 +430,38 @@ func (n *namespaces) GetSchemaValidationEnforced(namespace NameSpaceName) (bool, return strconv.ParseBool(string(r)) } -func (n *namespaces) SetSchemaAutoUpdateCompatibilityStrategy(namespace NameSpaceName, - strategy SchemaCompatibilityStrategy) error { +func (n *namespaces) SetSchemaAutoUpdateCompatibilityStrategy(namespace utils.NameSpaceName, + strategy utils.SchemaCompatibilityStrategy) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaAutoUpdateCompatibilityStrategy") return n.client.put(endpoint, strategy.String()) } -func (n *namespaces) GetSchemaAutoUpdateCompatibilityStrategy(namespace NameSpaceName) (SchemaCompatibilityStrategy, - error) { +func (n *namespaces) GetSchemaAutoUpdateCompatibilityStrategy(namespace utils.NameSpaceName) ( + utils.SchemaCompatibilityStrategy, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaAutoUpdateCompatibilityStrategy") b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) if err != nil { return "", err } - s, err := ParseSchemaAutoUpdateCompatibilityStrategy(strings.ReplaceAll(string(b), "\"", "")) + s, err := utils.ParseSchemaAutoUpdateCompatibilityStrategy(strings.ReplaceAll(string(b), "\"", "")) if err != nil { return "", err } return s, nil } -func (n *namespaces) ClearOffloadDeleteLag(namespace NameSpaceName) error { +func (n *namespaces) ClearOffloadDeleteLag(namespace utils.NameSpaceName) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") return n.client.delete(endpoint) } -func (n *namespaces) SetOffloadDeleteLag(namespace NameSpaceName, timeMs int64) error { +func (n *namespaces) SetOffloadDeleteLag(namespace utils.NameSpaceName, timeMs int64) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") return n.client.put(endpoint, timeMs) } -func (n *namespaces) GetOffloadDeleteLag(namespace NameSpaceName) (int64, error) { +func (n *namespaces) GetOffloadDeleteLag(namespace utils.NameSpaceName) (int64, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) if err != nil { @@ -466,12 +470,12 @@ func (n *namespaces) GetOffloadDeleteLag(namespace NameSpaceName) (int64, error) return strconv.ParseInt(string(b), 10, 64) } -func (n *namespaces) SetMaxConsumersPerSubscription(namespace NameSpaceName, max int) error { +func (n *namespaces) SetMaxConsumersPerSubscription(namespace utils.NameSpaceName, max int) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerSubscription") return n.client.post(endpoint, max) } -func (n *namespaces) GetMaxConsumersPerSubscription(namespace NameSpaceName) (int, error) { +func (n *namespaces) GetMaxConsumersPerSubscription(namespace utils.NameSpaceName) (int, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerSubscription") b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) if err != nil { @@ -480,12 +484,12 @@ func (n *namespaces) GetMaxConsumersPerSubscription(namespace NameSpaceName) (in return strconv.Atoi(string(b)) } -func (n *namespaces) SetOffloadThreshold(namespace NameSpaceName, threshold int64) error { +func (n *namespaces) SetOffloadThreshold(namespace utils.NameSpaceName, threshold int64) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadThreshold") return n.client.put(endpoint, threshold) } -func (n *namespaces) GetOffloadThreshold(namespace NameSpaceName) (int64, error) { +func (n *namespaces) GetOffloadThreshold(namespace utils.NameSpaceName) (int64, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadThreshold") b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) if err != nil { @@ -494,12 +498,12 @@ func (n *namespaces) GetOffloadThreshold(namespace NameSpaceName) (int64, error) return strconv.ParseInt(string(b), 10, 64) } -func (n *namespaces) SetMaxConsumersPerTopic(namespace NameSpaceName, max int) error { +func (n *namespaces) SetMaxConsumersPerTopic(namespace utils.NameSpaceName, max int) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerTopic") return n.client.post(endpoint, max) } -func (n *namespaces) GetMaxConsumersPerTopic(namespace NameSpaceName) (int, error) { +func (n *namespaces) GetMaxConsumersPerTopic(namespace utils.NameSpaceName) (int, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerTopic") b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) if err != nil { @@ -508,12 +512,12 @@ func (n *namespaces) GetMaxConsumersPerTopic(namespace NameSpaceName) (int, erro return strconv.Atoi(string(b)) } -func (n *namespaces) SetCompactionThreshold(namespace NameSpaceName, threshold int64) error { +func (n *namespaces) SetCompactionThreshold(namespace utils.NameSpaceName, threshold int64) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "compactionThreshold") return n.client.put(endpoint, threshold) } -func (n *namespaces) GetCompactionThreshold(namespace NameSpaceName) (int64, error) { +func (n *namespaces) GetCompactionThreshold(namespace utils.NameSpaceName) (int64, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "compactionThreshold") b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) if err != nil { @@ -522,12 +526,12 @@ func (n *namespaces) GetCompactionThreshold(namespace NameSpaceName) (int64, err return strconv.ParseInt(string(b), 10, 64) } -func (n *namespaces) SetMaxProducersPerTopic(namespace NameSpaceName, max int) error { +func (n *namespaces) SetMaxProducersPerTopic(namespace utils.NameSpaceName, max int) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxProducersPerTopic") return n.client.post(endpoint, max) } -func (n *namespaces) GetMaxProducersPerTopic(namespace NameSpaceName) (int, error) { +func (n *namespaces) GetMaxProducersPerTopic(namespace utils.NameSpaceName) (int, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxProducersPerTopic") b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) if err != nil { @@ -538,7 +542,7 @@ func (n *namespaces) GetMaxProducersPerTopic(namespace NameSpaceName) (int, erro func (n *namespaces) GetNamespaceReplicationClusters(namespace string) ([]string, error) { var data []string - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return nil, err } @@ -548,7 +552,7 @@ func (n *namespaces) GetNamespaceReplicationClusters(namespace string) ([]string } func (n *namespaces) SetNamespaceReplicationClusters(namespace string, clusterIds []string) error { - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -557,7 +561,7 @@ func (n *namespaces) SetNamespaceReplicationClusters(namespace string, clusterId } func (n *namespaces) SetNamespaceAntiAffinityGroup(namespace string, namespaceAntiAffinityGroup string) error { - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -576,7 +580,7 @@ func (n *namespaces) GetAntiAffinityNamespaces(tenant, cluster, namespaceAntiAff } func (n *namespaces) GetNamespaceAntiAffinityGroup(namespace string) (string, error) { - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return "", err } @@ -586,7 +590,7 @@ func (n *namespaces) GetNamespaceAntiAffinityGroup(namespace string) (string, er } func (n *namespaces) DeleteNamespaceAntiAffinityGroup(namespace string) error { - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -595,7 +599,7 @@ func (n *namespaces) DeleteNamespaceAntiAffinityGroup(namespace string) error { } func (n *namespaces) SetDeduplicationStatus(namespace string, enableDeduplication bool) error { - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -603,8 +607,8 @@ func (n *namespaces) SetDeduplicationStatus(namespace string, enableDeduplicatio return n.client.post(endpoint, enableDeduplication) } -func (n *namespaces) SetPersistence(namespace string, persistence PersistencePolicies) error { - nsName, err := GetNamespaceName(namespace) +func (n *namespaces) SetPersistence(namespace string, persistence utils.PersistencePolicies) error { + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -612,8 +616,8 @@ func (n *namespaces) SetPersistence(namespace string, persistence PersistencePol return n.client.post(endpoint, &persistence) } -func (n *namespaces) SetBookieAffinityGroup(namespace string, bookieAffinityGroup BookieAffinityGroupData) error { - nsName, err := GetNamespaceName(namespace) +func (n *namespaces) SetBookieAffinityGroup(namespace string, bookieAffinityGroup utils.BookieAffinityGroupData) error { + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -622,7 +626,7 @@ func (n *namespaces) SetBookieAffinityGroup(namespace string, bookieAffinityGrou } func (n *namespaces) DeleteBookieAffinityGroup(namespace string) error { - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -630,9 +634,9 @@ func (n *namespaces) DeleteBookieAffinityGroup(namespace string) error { return n.client.delete(endpoint) } -func (n *namespaces) GetBookieAffinityGroup(namespace string) (*BookieAffinityGroupData, error) { - var data BookieAffinityGroupData - nsName, err := GetNamespaceName(namespace) +func (n *namespaces) GetBookieAffinityGroup(namespace string) (*utils.BookieAffinityGroupData, error) { + var data utils.BookieAffinityGroupData + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return nil, err } @@ -641,9 +645,9 @@ func (n *namespaces) GetBookieAffinityGroup(namespace string) (*BookieAffinityGr return &data, err } -func (n *namespaces) GetPersistence(namespace string) (*PersistencePolicies, error) { - var persistence PersistencePolicies - nsName, err := GetNamespaceName(namespace) +func (n *namespaces) GetPersistence(namespace string) (*utils.PersistencePolicies, error) { + var persistence utils.PersistencePolicies + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return nil, err } @@ -653,7 +657,7 @@ func (n *namespaces) GetPersistence(namespace string) (*PersistencePolicies, err } func (n *namespaces) Unload(namespace string) error { - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -662,7 +666,7 @@ func (n *namespaces) Unload(namespace string) error { } func (n *namespaces) UnloadNamespaceBundle(namespace, bundle string) error { - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -671,7 +675,7 @@ func (n *namespaces) UnloadNamespaceBundle(namespace, bundle string) error { } func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitBundles bool) error { - nsName, err := GetNamespaceName(namespace) + nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } @@ -682,14 +686,15 @@ func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitB return n.client.putWithQueryParams(endpoint, "", nil, params) } -func (n *namespaces) GetNamespacePermissions(namespace NameSpaceName) (map[string][]AuthAction, error) { +func (n *namespaces) GetNamespacePermissions(namespace utils.NameSpaceName) (map[string][]common.AuthAction, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions") - var permissions map[string][]AuthAction + var permissions map[string][]common.AuthAction err := n.client.get(endpoint, &permissions) return permissions, err } -func (n *namespaces) GrantNamespacePermission(namespace NameSpaceName, role string, action []AuthAction) error { +func (n *namespaces) GrantNamespacePermission(namespace utils.NameSpaceName, role string, + action []common.AuthAction) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", role) s := make([]string, 0) for _, v := range action { @@ -698,107 +703,108 @@ func (n *namespaces) GrantNamespacePermission(namespace NameSpaceName, role stri return n.client.post(endpoint, s) } -func (n *namespaces) RevokeNamespacePermission(namespace NameSpaceName, role string) error { +func (n *namespaces) RevokeNamespacePermission(namespace utils.NameSpaceName, role string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", role) return n.client.delete(endpoint) } -func (n *namespaces) GrantSubPermission(namespace NameSpaceName, sName string, roles []string) error { +func (n *namespaces) GrantSubPermission(namespace utils.NameSpaceName, sName string, roles []string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", "subscription", sName) return n.client.post(endpoint, roles) } -func (n *namespaces) RevokeSubPermission(namespace NameSpaceName, sName, role string) error { +func (n *namespaces) RevokeSubPermission(namespace utils.NameSpaceName, sName, role string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", "subscription", sName, role) return n.client.delete(endpoint) } -func (n *namespaces) SetSubscriptionAuthMode(namespace NameSpaceName, mode SubscriptionAuthMode) error { +func (n *namespaces) SetSubscriptionAuthMode(namespace utils.NameSpaceName, mode utils.SubscriptionAuthMode) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionAuthMode") return n.client.post(endpoint, mode.String()) } -func (n *namespaces) SetEncryptionRequiredStatus(namespace NameSpaceName, encrypt bool) error { +func (n *namespaces) SetEncryptionRequiredStatus(namespace utils.NameSpaceName, encrypt bool) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "encryptionRequired") return n.client.post(endpoint, strconv.FormatBool(encrypt)) } -func (n *namespaces) UnsubscribeNamespace(namespace NameSpaceName, sName string) error { +func (n *namespaces) UnsubscribeNamespace(namespace utils.NameSpaceName, sName string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "unsubscribe", url.QueryEscape(sName)) return n.client.post(endpoint, "") } -func (n *namespaces) UnsubscribeNamespaceBundle(namespace NameSpaceName, bundle, sName string) error { +func (n *namespaces) UnsubscribeNamespaceBundle(namespace utils.NameSpaceName, bundle, sName string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "unsubscribe", url.QueryEscape(sName)) return n.client.post(endpoint, "") } -func (n *namespaces) ClearNamespaceBundleBacklogForSubscription(namespace NameSpaceName, bundle, sName string) error { +func (n *namespaces) ClearNamespaceBundleBacklogForSubscription(namespace utils.NameSpaceName, + bundle, sName string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog", url.QueryEscape(sName)) return n.client.post(endpoint, "") } -func (n *namespaces) ClearNamespaceBundleBacklog(namespace NameSpaceName, bundle string) error { +func (n *namespaces) ClearNamespaceBundleBacklog(namespace utils.NameSpaceName, bundle string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog") return n.client.post(endpoint, "") } -func (n *namespaces) ClearNamespaceBacklogForSubscription(namespace NameSpaceName, sName string) error { +func (n *namespaces) ClearNamespaceBacklogForSubscription(namespace utils.NameSpaceName, sName string) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "clearBacklog", url.QueryEscape(sName)) return n.client.post(endpoint, "") } -func (n *namespaces) ClearNamespaceBacklog(namespace NameSpaceName) error { +func (n *namespaces) ClearNamespaceBacklog(namespace utils.NameSpaceName) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "clearBacklog") return n.client.post(endpoint, "") } -func (n *namespaces) SetReplicatorDispatchRate(namespace NameSpaceName, rate DispatchRate) error { +func (n *namespaces) SetReplicatorDispatchRate(namespace utils.NameSpaceName, rate utils.DispatchRate) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "replicatorDispatchRate") return n.client.post(endpoint, rate) } -func (n *namespaces) GetReplicatorDispatchRate(namespace NameSpaceName) (DispatchRate, error) { +func (n *namespaces) GetReplicatorDispatchRate(namespace utils.NameSpaceName) (utils.DispatchRate, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "replicatorDispatchRate") - var rate DispatchRate + var rate utils.DispatchRate err := n.client.get(endpoint, &rate) return rate, err } -func (n *namespaces) SetSubscriptionDispatchRate(namespace NameSpaceName, rate DispatchRate) error { +func (n *namespaces) SetSubscriptionDispatchRate(namespace utils.NameSpaceName, rate utils.DispatchRate) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionDispatchRate") return n.client.post(endpoint, rate) } -func (n *namespaces) GetSubscriptionDispatchRate(namespace NameSpaceName) (DispatchRate, error) { +func (n *namespaces) GetSubscriptionDispatchRate(namespace utils.NameSpaceName) (utils.DispatchRate, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionDispatchRate") - var rate DispatchRate + var rate utils.DispatchRate err := n.client.get(endpoint, &rate) return rate, err } -func (n *namespaces) SetSubscribeRate(namespace NameSpaceName, rate SubscribeRate) error { +func (n *namespaces) SetSubscribeRate(namespace utils.NameSpaceName, rate utils.SubscribeRate) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscribeRate") return n.client.post(endpoint, rate) } -func (n *namespaces) GetSubscribeRate(namespace NameSpaceName) (SubscribeRate, error) { +func (n *namespaces) GetSubscribeRate(namespace utils.NameSpaceName) (utils.SubscribeRate, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscribeRate") - var rate SubscribeRate + var rate utils.SubscribeRate err := n.client.get(endpoint, &rate) return rate, err } -func (n *namespaces) SetDispatchRate(namespace NameSpaceName, rate DispatchRate) error { +func (n *namespaces) SetDispatchRate(namespace utils.NameSpaceName, rate utils.DispatchRate) error { endpoint := n.client.endpoint(n.basePath, namespace.String(), "dispatchRate") return n.client.post(endpoint, rate) } -func (n *namespaces) GetDispatchRate(namespace NameSpaceName) (DispatchRate, error) { +func (n *namespaces) GetDispatchRate(namespace utils.NameSpaceName) (utils.DispatchRate, error) { endpoint := n.client.endpoint(n.basePath, namespace.String(), "dispatchRate") - var rate DispatchRate + var rate utils.DispatchRate err := n.client.get(endpoint, &rate) return rate, err } diff --git a/pulsaradmin/pkg/pulsar/ns_isolation_policy.go b/pulsaradmin/pkg/pulsar/ns_isolation_policy.go index 99c573fde0..faaf70554d 100644 --- a/pulsaradmin/pkg/pulsar/ns_isolation_policy.go +++ b/pulsaradmin/pkg/pulsar/ns_isolation_policy.go @@ -17,24 +17,28 @@ package pulsar +import ( + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" +) + type NsIsolationPolicy interface { // Create a namespace isolation policy for a cluster - CreateNamespaceIsolationPolicy(cluster, policyName string, namespaceIsolationData NamespaceIsolationData) error + CreateNamespaceIsolationPolicy(cluster, policyName string, namespaceIsolationData utils.NamespaceIsolationData) error // Delete a namespace isolation policy for a cluster DeleteNamespaceIsolationPolicy(cluster, policyName string) error // Get a single namespace isolation policy for a cluster - GetNamespaceIsolationPolicy(cluster, policyName string) (*NamespaceIsolationData, error) + GetNamespaceIsolationPolicy(cluster, policyName string) (*utils.NamespaceIsolationData, error) // Get the namespace isolation policies of a cluster - GetNamespaceIsolationPolicies(cluster string) (map[string]NamespaceIsolationData, error) + GetNamespaceIsolationPolicies(cluster string) (map[string]utils.NamespaceIsolationData, error) // Returns list of active brokers with namespace-isolation policies attached to it. - GetBrokersWithNamespaceIsolationPolicy(cluster string) ([]BrokerNamespaceIsolationData, error) + GetBrokersWithNamespaceIsolationPolicy(cluster string) ([]utils.BrokerNamespaceIsolationData, error) // Returns active broker with namespace-isolation policies attached to it. - GetBrokerWithNamespaceIsolationPolicy(cluster, broker string) (*BrokerNamespaceIsolationData, error) + GetBrokerWithNamespaceIsolationPolicy(cluster, broker string) (*utils.BrokerNamespaceIsolationData, error) } type nsIsolationPolicy struct { @@ -50,12 +54,12 @@ func (c *client) NsIsolationPolicy() NsIsolationPolicy { } func (n *nsIsolationPolicy) CreateNamespaceIsolationPolicy(cluster, policyName string, - namespaceIsolationData NamespaceIsolationData) error { + namespaceIsolationData utils.NamespaceIsolationData) error { return n.setNamespaceIsolationPolicy(cluster, policyName, namespaceIsolationData) } func (n *nsIsolationPolicy) setNamespaceIsolationPolicy(cluster, policyName string, - namespaceIsolationData NamespaceIsolationData) error { + namespaceIsolationData utils.NamespaceIsolationData) error { endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", policyName) return n.client.post(endpoint, &namespaceIsolationData) } @@ -65,9 +69,10 @@ func (n *nsIsolationPolicy) DeleteNamespaceIsolationPolicy(cluster, policyName s return n.client.delete(endpoint) } -func (n *nsIsolationPolicy) GetNamespaceIsolationPolicy(cluster, policyName string) (*NamespaceIsolationData, error) { +func (n *nsIsolationPolicy) GetNamespaceIsolationPolicy(cluster, policyName string) ( + *utils.NamespaceIsolationData, error) { endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", policyName) - var nsIsolationData NamespaceIsolationData + var nsIsolationData utils.NamespaceIsolationData err := n.client.get(endpoint, &nsIsolationData) if err != nil { return nil, err @@ -75,9 +80,10 @@ func (n *nsIsolationPolicy) GetNamespaceIsolationPolicy(cluster, policyName stri return &nsIsolationData, nil } -func (n *nsIsolationPolicy) GetNamespaceIsolationPolicies(cluster string) (map[string]NamespaceIsolationData, error) { +func (n *nsIsolationPolicy) GetNamespaceIsolationPolicies(cluster string) ( + map[string]utils.NamespaceIsolationData, error) { endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies") - var tmpMap map[string]NamespaceIsolationData + var tmpMap map[string]utils.NamespaceIsolationData err := n.client.get(endpoint, &tmpMap) if err != nil { return nil, err @@ -86,9 +92,9 @@ func (n *nsIsolationPolicy) GetNamespaceIsolationPolicies(cluster string) (map[s } func (n *nsIsolationPolicy) GetBrokersWithNamespaceIsolationPolicy(cluster string) ( - []BrokerNamespaceIsolationData, error) { + []utils.BrokerNamespaceIsolationData, error) { endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", "brokers") - var res []BrokerNamespaceIsolationData + var res []utils.BrokerNamespaceIsolationData err := n.client.get(endpoint, &res) if err != nil { return nil, err @@ -97,9 +103,9 @@ func (n *nsIsolationPolicy) GetBrokersWithNamespaceIsolationPolicy(cluster strin } func (n *nsIsolationPolicy) GetBrokerWithNamespaceIsolationPolicy(cluster, - broker string) (*BrokerNamespaceIsolationData, error) { + broker string) (*utils.BrokerNamespaceIsolationData, error) { endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", "brokers", broker) - var brokerNamespaceIsolationData BrokerNamespaceIsolationData + var brokerNamespaceIsolationData utils.BrokerNamespaceIsolationData err := n.client.get(endpoint, &brokerNamespaceIsolationData) if err != nil { return nil, err diff --git a/pulsaradmin/pkg/pulsar/resource_quotas.go b/pulsaradmin/pkg/pulsar/resource_quotas.go index 39afcc136d..50f2bd3685 100644 --- a/pulsaradmin/pkg/pulsar/resource_quotas.go +++ b/pulsaradmin/pkg/pulsar/resource_quotas.go @@ -17,18 +17,22 @@ package pulsar +import ( + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" +) + type ResourceQuotas interface { // Get default resource quota for new resource bundles. - GetDefaultResourceQuota() (*ResourceQuota, error) + GetDefaultResourceQuota() (*utils.ResourceQuota, error) // Set default resource quota for new namespace bundles. - SetDefaultResourceQuota(quota ResourceQuota) error + SetDefaultResourceQuota(quota utils.ResourceQuota) error // Get resource quota of a namespace bundle. - GetNamespaceBundleResourceQuota(namespace, bundle string) (*ResourceQuota, error) + GetNamespaceBundleResourceQuota(namespace, bundle string) (*utils.ResourceQuota, error) // Set resource quota for a namespace bundle. - SetNamespaceBundleResourceQuota(namespace, bundle string, quota ResourceQuota) error + SetNamespaceBundleResourceQuota(namespace, bundle string, quota utils.ResourceQuota) error // Reset resource quota for a namespace bundle to default value. ResetNamespaceBundleResourceQuota(namespace, bundle string) error @@ -46,9 +50,9 @@ func (c *client) ResourceQuotas() ResourceQuotas { } } -func (r *resource) GetDefaultResourceQuota() (*ResourceQuota, error) { +func (r *resource) GetDefaultResourceQuota() (*utils.ResourceQuota, error) { endpoint := r.client.endpoint(r.basePath) - var quota ResourceQuota + var quota utils.ResourceQuota err := r.client.get(endpoint, "a) if err != nil { return nil, err @@ -56,14 +60,14 @@ func (r *resource) GetDefaultResourceQuota() (*ResourceQuota, error) { return "a, nil } -func (r *resource) SetDefaultResourceQuota(quota ResourceQuota) error { +func (r *resource) SetDefaultResourceQuota(quota utils.ResourceQuota) error { endpoint := r.client.endpoint(r.basePath) return r.client.post(endpoint, "a) } -func (r *resource) GetNamespaceBundleResourceQuota(namespace, bundle string) (*ResourceQuota, error) { +func (r *resource) GetNamespaceBundleResourceQuota(namespace, bundle string) (*utils.ResourceQuota, error) { endpoint := r.client.endpoint(r.basePath, namespace, bundle) - var quota ResourceQuota + var quota utils.ResourceQuota err := r.client.get(endpoint, "a) if err != nil { return nil, err @@ -71,7 +75,7 @@ func (r *resource) GetNamespaceBundleResourceQuota(namespace, bundle string) (*R return "a, nil } -func (r *resource) SetNamespaceBundleResourceQuota(namespace, bundle string, quota ResourceQuota) error { +func (r *resource) SetNamespaceBundleResourceQuota(namespace, bundle string, quota utils.ResourceQuota) error { endpoint := r.client.endpoint(r.basePath, namespace, bundle) return r.client.post(endpoint, "a) } diff --git a/pulsaradmin/pkg/pulsar/schema.go b/pulsaradmin/pkg/pulsar/schema.go index 9fbd7cbb4c..55e4e10bb5 100644 --- a/pulsaradmin/pkg/pulsar/schema.go +++ b/pulsaradmin/pkg/pulsar/schema.go @@ -20,24 +20,26 @@ package pulsar import ( "fmt" "strconv" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) // Schema is admin interface for schema management type Schema interface { // GetSchemaInfo retrieves the latest schema of a topic - GetSchemaInfo(topic string) (*SchemaInfo, error) + GetSchemaInfo(topic string) (*utils.SchemaInfo, error) // GetSchemaInfoWithVersion retrieves the latest schema with version of a topic - GetSchemaInfoWithVersion(topic string) (*SchemaInfoWithVersion, error) + GetSchemaInfoWithVersion(topic string) (*utils.SchemaInfoWithVersion, error) // GetSchemaInfoByVersion retrieves the schema of a topic at a given version - GetSchemaInfoByVersion(topic string, version int64) (*SchemaInfo, error) + GetSchemaInfoByVersion(topic string, version int64) (*utils.SchemaInfo, error) // DeleteSchema deletes the schema associated with a given topic DeleteSchema(topic string) error // CreateSchemaByPayload creates a schema for a given topic - CreateSchemaByPayload(topic string, schemaPayload PostSchemaPayload) error + CreateSchemaByPayload(topic string, schemaPayload utils.PostSchemaPayload) error } type schemas struct { @@ -53,30 +55,31 @@ func (c *client) Schemas() Schema { } } -func (s *schemas) GetSchemaInfo(topic string) (*SchemaInfo, error) { - topicName, err := GetTopicName(topic) +func (s *schemas) GetSchemaInfo(topic string) (*utils.SchemaInfo, error) { + topicName, err := utils.GetTopicName(topic) if err != nil { return nil, err } - var response GetSchemaResponse - endpoint := s.client.endpoint(s.basePath, topicName.tenant, topicName.namespace, topicName.GetEncodedTopic(), "schema") + var response utils.GetSchemaResponse + endpoint := s.client.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), + topicName.GetEncodedTopic(), "schema") err = s.client.get(endpoint, &response) if err != nil { return nil, err } - info := convertGetSchemaResponseToSchemaInfo(topicName, response) + info := utils.ConvertGetSchemaResponseToSchemaInfo(topicName, response) return info, nil } -func (s *schemas) GetSchemaInfoWithVersion(topic string) (*SchemaInfoWithVersion, error) { - topicName, err := GetTopicName(topic) +func (s *schemas) GetSchemaInfoWithVersion(topic string) (*utils.SchemaInfoWithVersion, error) { + topicName, err := utils.GetTopicName(topic) if err != nil { return nil, err } - var response GetSchemaResponse - endpoint := s.client.endpoint(s.basePath, topicName.tenant, topicName.namespace, + var response utils.GetSchemaResponse + endpoint := s.client.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetEncodedTopic(), "schema") err = s.client.get(endpoint, &response) @@ -85,18 +88,18 @@ func (s *schemas) GetSchemaInfoWithVersion(topic string) (*SchemaInfoWithVersion return nil, err } - info := convertGetSchemaResponseToSchemaInfoWithVersion(topicName, response) + info := utils.ConvertGetSchemaResponseToSchemaInfoWithVersion(topicName, response) return info, nil } -func (s *schemas) GetSchemaInfoByVersion(topic string, version int64) (*SchemaInfo, error) { - topicName, err := GetTopicName(topic) +func (s *schemas) GetSchemaInfoByVersion(topic string, version int64) (*utils.SchemaInfo, error) { + topicName, err := utils.GetTopicName(topic) if err != nil { return nil, err } - var response GetSchemaResponse - endpoint := s.client.endpoint(s.basePath, topicName.tenant, topicName.namespace, topicName.GetEncodedTopic(), + var response utils.GetSchemaResponse + endpoint := s.client.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetEncodedTopic(), "schema", strconv.FormatInt(version, 10)) err = s.client.get(endpoint, &response) @@ -104,17 +107,17 @@ func (s *schemas) GetSchemaInfoByVersion(topic string, version int64) (*SchemaIn return nil, err } - info := convertGetSchemaResponseToSchemaInfo(topicName, response) + info := utils.ConvertGetSchemaResponseToSchemaInfo(topicName, response) return info, nil } func (s *schemas) DeleteSchema(topic string) error { - topicName, err := GetTopicName(topic) + topicName, err := utils.GetTopicName(topic) if err != nil { return err } - endpoint := s.client.endpoint(s.basePath, topicName.tenant, topicName.namespace, + endpoint := s.client.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetEncodedTopic(), "schema") fmt.Println(endpoint) @@ -122,13 +125,13 @@ func (s *schemas) DeleteSchema(topic string) error { return s.client.delete(endpoint) } -func (s *schemas) CreateSchemaByPayload(topic string, schemaPayload PostSchemaPayload) error { - topicName, err := GetTopicName(topic) +func (s *schemas) CreateSchemaByPayload(topic string, schemaPayload utils.PostSchemaPayload) error { + topicName, err := utils.GetTopicName(topic) if err != nil { return err } - endpoint := s.client.endpoint(s.basePath, topicName.tenant, topicName.namespace, + endpoint := s.client.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetEncodedTopic(), "schema") return s.client.post(endpoint, &schemaPayload) diff --git a/pulsaradmin/pkg/pulsar/sinks.go b/pulsaradmin/pkg/pulsar/sinks.go index 22b8cb454d..c9033cec5e 100644 --- a/pulsaradmin/pkg/pulsar/sinks.go +++ b/pulsaradmin/pkg/pulsar/sinks.go @@ -27,6 +27,8 @@ import ( "os" "path/filepath" "strings" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) // Sinks is admin interface for sinks management @@ -35,28 +37,28 @@ type Sinks interface { ListSinks(tenant, namespace string) ([]string, error) // GetSink returns the configuration for the specified sink - GetSink(tenant, namespace, Sink string) (SinkConfig, error) + GetSink(tenant, namespace, Sink string) (utils.SinkConfig, error) // CreateSink creates a new sink - CreateSink(config *SinkConfig, fileName string) error + CreateSink(config *utils.SinkConfig, fileName string) error // CreateSinkWithURL creates a new sink by providing url from which fun-pkg can be downloaded. supported url: http/file - CreateSinkWithURL(config *SinkConfig, pkgURL string) error + CreateSinkWithURL(config *utils.SinkConfig, pkgURL string) error // UpdateSink updates the configuration for a sink. - UpdateSink(config *SinkConfig, fileName string, options *UpdateOptions) error + UpdateSink(config *utils.SinkConfig, fileName string, options *utils.UpdateOptions) error // UpdateSinkWithURL updates a sink by providing url from which fun-pkg can be downloaded. supported url: http/file - UpdateSinkWithURL(config *SinkConfig, pkgURL string, options *UpdateOptions) error + UpdateSinkWithURL(config *utils.SinkConfig, pkgURL string, options *utils.UpdateOptions) error // DeleteSink deletes an existing sink DeleteSink(tenant, namespace, Sink string) error // GetSinkStatus returns the current status of a sink. - GetSinkStatus(tenant, namespace, Sink string) (SinkStatus, error) + GetSinkStatus(tenant, namespace, Sink string) (utils.SinkStatus, error) // GetSinkStatusWithID returns the current status of a sink instance. - GetSinkStatusWithID(tenant, namespace, Sink string, id int) (SinkInstanceStatusData, error) + GetSinkStatusWithID(tenant, namespace, Sink string, id int) (utils.SinkInstanceStatusData, error) // RestartSink restarts all sink instances RestartSink(tenant, namespace, Sink string) error @@ -77,7 +79,7 @@ type Sinks interface { StartSinkWithID(tenant, namespace, Sink string, id int) error // GetBuiltInSinks fetches a list of supported Pulsar IO sinks currently running in cluster mode - GetBuiltInSinks() ([]*ConnectorDefinition, error) + GetBuiltInSinks() ([]*utils.ConnectorDefinition, error) // ReloadBuiltInSinks reload the available built-in connectors, include Source and Sink ReloadBuiltInSinks() error @@ -117,14 +119,14 @@ func (s *sinks) ListSinks(tenant, namespace string) ([]string, error) { return sinks, err } -func (s *sinks) GetSink(tenant, namespace, sink string) (SinkConfig, error) { - var sinkConfig SinkConfig +func (s *sinks) GetSink(tenant, namespace, sink string) (utils.SinkConfig, error) { + var sinkConfig utils.SinkConfig endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) err := s.client.get(endpoint, &sinkConfig) return sinkConfig, err } -func (s *sinks) CreateSink(config *SinkConfig, fileName string) error { +func (s *sinks) CreateSink(config *utils.SinkConfig, fileName string) error { endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes @@ -182,7 +184,7 @@ func (s *sinks) CreateSink(config *SinkConfig, fileName string) error { return nil } -func (s *sinks) CreateSinkWithURL(config *SinkConfig, pkgURL string) error { +func (s *sinks) CreateSinkWithURL(config *utils.SinkConfig, pkgURL string) error { endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -227,7 +229,7 @@ func (s *sinks) CreateSinkWithURL(config *SinkConfig, pkgURL string) error { return nil } -func (s *sinks) UpdateSink(config *SinkConfig, fileName string, updateOptions *UpdateOptions) error { +func (s *sinks) UpdateSink(config *utils.SinkConfig, fileName string, updateOptions *utils.UpdateOptions) error { endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -302,7 +304,7 @@ func (s *sinks) UpdateSink(config *SinkConfig, fileName string, updateOptions *U return nil } -func (s *sinks) UpdateSinkWithURL(config *SinkConfig, pkgURL string, updateOptions *UpdateOptions) error { +func (s *sinks) UpdateSinkWithURL(config *utils.SinkConfig, pkgURL string, updateOptions *utils.UpdateOptions) error { endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -371,15 +373,15 @@ func (s *sinks) DeleteSink(tenant, namespace, sink string) error { return s.client.delete(endpoint) } -func (s *sinks) GetSinkStatus(tenant, namespace, sink string) (SinkStatus, error) { - var sinkStatus SinkStatus +func (s *sinks) GetSinkStatus(tenant, namespace, sink string) (utils.SinkStatus, error) { + var sinkStatus utils.SinkStatus endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) err := s.client.get(endpoint+"/status", &sinkStatus) return sinkStatus, err } -func (s *sinks) GetSinkStatusWithID(tenant, namespace, sink string, id int) (SinkInstanceStatusData, error) { - var sinkInstanceStatusData SinkInstanceStatusData +func (s *sinks) GetSinkStatusWithID(tenant, namespace, sink string, id int) (utils.SinkInstanceStatusData, error) { + var sinkInstanceStatusData utils.SinkInstanceStatusData instanceID := fmt.Sprintf("%d", id) endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink, instanceID) err := s.client.get(endpoint+"/status", &sinkInstanceStatusData) @@ -422,8 +424,8 @@ func (s *sinks) StartSinkWithID(tenant, namespace, sink string, instanceID int) return s.client.post(endpoint+"/start", "") } -func (s *sinks) GetBuiltInSinks() ([]*ConnectorDefinition, error) { - var connectorDefinition []*ConnectorDefinition +func (s *sinks) GetBuiltInSinks() ([]*utils.ConnectorDefinition, error) { + var connectorDefinition []*utils.ConnectorDefinition endpoint := s.client.endpoint(s.basePath, "builtinSinks") err := s.client.get(endpoint, &connectorDefinition) return connectorDefinition, err diff --git a/pulsaradmin/pkg/pulsar/sources.go b/pulsaradmin/pkg/pulsar/sources.go index 4a823b8d6c..1c5cac075a 100644 --- a/pulsaradmin/pkg/pulsar/sources.go +++ b/pulsaradmin/pkg/pulsar/sources.go @@ -27,6 +27,8 @@ import ( "os" "path/filepath" "strings" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) // Sources is admin interface for sources management @@ -35,29 +37,29 @@ type Sources interface { ListSources(tenant, namespace string) ([]string, error) // GetSource return the configuration for the specified source - GetSource(tenant, namespace, source string) (SourceConfig, error) + GetSource(tenant, namespace, source string) (utils.SourceConfig, error) // CreateSource creates a new source - CreateSource(config *SourceConfig, fileName string) error + CreateSource(config *utils.SourceConfig, fileName string) error // CreateSourceWithURL creates a new source by providing url from which fun-pkg can be downloaded. // supported url: http/file - CreateSourceWithURL(config *SourceConfig, pkgURL string) error + CreateSourceWithURL(config *utils.SourceConfig, pkgURL string) error // UpdateSource updates the configuration for a source. - UpdateSource(config *SourceConfig, fileName string, options *UpdateOptions) error + UpdateSource(config *utils.SourceConfig, fileName string, options *utils.UpdateOptions) error // UpdateSourceWithURL updates a source by providing url from which fun-pkg can be downloaded. supported url: http/file - UpdateSourceWithURL(config *SourceConfig, pkgURL string, options *UpdateOptions) error + UpdateSourceWithURL(config *utils.SourceConfig, pkgURL string, options *utils.UpdateOptions) error // DeleteSource deletes an existing source DeleteSource(tenant, namespace, source string) error // GetSourceStatus returns the current status of a source. - GetSourceStatus(tenant, namespace, source string) (SourceStatus, error) + GetSourceStatus(tenant, namespace, source string) (utils.SourceStatus, error) // GetSourceStatusWithID returns the current status of a source instance. - GetSourceStatusWithID(tenant, namespace, source string, id int) (SourceInstanceStatusData, error) + GetSourceStatusWithID(tenant, namespace, source string, id int) (utils.SourceInstanceStatusData, error) // RestartSource restarts all source instances RestartSource(tenant, namespace, source string) error @@ -78,7 +80,7 @@ type Sources interface { StartSourceWithID(tenant, namespace, source string, id int) error // GetBuiltInSources fetches a list of supported Pulsar IO sources currently running in cluster mode - GetBuiltInSources() ([]*ConnectorDefinition, error) + GetBuiltInSources() ([]*utils.ConnectorDefinition, error) // ReloadBuiltInSources reloads the available built-in connectors, include Source and Sink ReloadBuiltInSources() error @@ -118,14 +120,14 @@ func (s *sources) ListSources(tenant, namespace string) ([]string, error) { return sources, err } -func (s *sources) GetSource(tenant, namespace, source string) (SourceConfig, error) { - var sourceConfig SourceConfig +func (s *sources) GetSource(tenant, namespace, source string) (utils.SourceConfig, error) { + var sourceConfig utils.SourceConfig endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) err := s.client.get(endpoint, &sourceConfig) return sourceConfig, err } -func (s *sources) CreateSource(config *SourceConfig, fileName string) error { +func (s *sources) CreateSource(config *utils.SourceConfig, fileName string) error { endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes @@ -183,7 +185,7 @@ func (s *sources) CreateSource(config *SourceConfig, fileName string) error { return nil } -func (s *sources) CreateSourceWithURL(config *SourceConfig, pkgURL string) error { +func (s *sources) CreateSourceWithURL(config *utils.SourceConfig, pkgURL string) error { endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -228,7 +230,7 @@ func (s *sources) CreateSourceWithURL(config *SourceConfig, pkgURL string) error return nil } -func (s *sources) UpdateSource(config *SourceConfig, fileName string, updateOptions *UpdateOptions) error { +func (s *sources) UpdateSource(config *utils.SourceConfig, fileName string, updateOptions *utils.UpdateOptions) error { endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -303,7 +305,8 @@ func (s *sources) UpdateSource(config *SourceConfig, fileName string, updateOpti return nil } -func (s *sources) UpdateSourceWithURL(config *SourceConfig, pkgURL string, updateOptions *UpdateOptions) error { +func (s *sources) UpdateSourceWithURL(config *utils.SourceConfig, pkgURL string, + updateOptions *utils.UpdateOptions) error { endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -372,15 +375,16 @@ func (s *sources) DeleteSource(tenant, namespace, source string) error { return s.client.delete(endpoint) } -func (s *sources) GetSourceStatus(tenant, namespace, source string) (SourceStatus, error) { - var sourceStatus SourceStatus +func (s *sources) GetSourceStatus(tenant, namespace, source string) (utils.SourceStatus, error) { + var sourceStatus utils.SourceStatus endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) err := s.client.get(endpoint+"/status", &sourceStatus) return sourceStatus, err } -func (s *sources) GetSourceStatusWithID(tenant, namespace, source string, id int) (SourceInstanceStatusData, error) { - var sourceInstanceStatusData SourceInstanceStatusData +func (s *sources) GetSourceStatusWithID(tenant, namespace, source string, id int) ( + utils.SourceInstanceStatusData, error) { + var sourceInstanceStatusData utils.SourceInstanceStatusData instanceID := fmt.Sprintf("%d", id) endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, instanceID) err := s.client.get(endpoint+"/status", &sourceInstanceStatusData) @@ -423,8 +427,8 @@ func (s *sources) StartSourceWithID(tenant, namespace, source string, instanceID return s.client.post(endpoint+"/start", "") } -func (s *sources) GetBuiltInSources() ([]*ConnectorDefinition, error) { - var connectorDefinition []*ConnectorDefinition +func (s *sources) GetBuiltInSources() ([]*utils.ConnectorDefinition, error) { + var connectorDefinition []*utils.ConnectorDefinition endpoint := s.client.endpoint(s.basePath, "builtinsources") err := s.client.get(endpoint, &connectorDefinition) return connectorDefinition, err diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/pulsar/subscription.go index 76d62d6f61..5e59686bf1 100644 --- a/pulsaradmin/pkg/pulsar/subscription.go +++ b/pulsaradmin/pkg/pulsar/subscription.go @@ -28,45 +28,46 @@ import ( "strings" "github.com/golang/protobuf/proto" + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) // Subscriptions is admin interface for subscriptions management type Subscriptions interface { // Create a new subscription on a topic - Create(TopicName, string, MessageID) error + Create(utils.TopicName, string, utils.MessageID) error // Delete a subscription. // Delete a persistent subscription from a topic. There should not be any active consumers on the subscription - Delete(TopicName, string) error + Delete(utils.TopicName, string) error // List returns the list of subscriptions - List(TopicName) ([]string, error) + List(utils.TopicName) ([]string, error) // ResetCursorToMessageID resets cursor position on a topic subscription // @param // messageID reset subscription to messageId (or previous nearest messageId if given messageId is not valid) - ResetCursorToMessageID(TopicName, string, MessageID) error + ResetCursorToMessageID(utils.TopicName, string, utils.MessageID) error // ResetCursorToTimestamp resets cursor position on a topic subscription // @param // time reset subscription to position closest to time in ms since epoch - ResetCursorToTimestamp(TopicName, string, int64) error + ResetCursorToTimestamp(utils.TopicName, string, int64) error // ClearBacklog skips all messages on a topic subscription - ClearBacklog(TopicName, string) error + ClearBacklog(utils.TopicName, string) error // SkipMessages skips messages on a topic subscription - SkipMessages(TopicName, string, int64) error + SkipMessages(utils.TopicName, string, int64) error // ExpireMessages expires all messages older than given N (expireTimeInSeconds) seconds for a given subscription - ExpireMessages(TopicName, string, int64) error + ExpireMessages(utils.TopicName, string, int64) error // ExpireAllMessages expires all messages older than given N (expireTimeInSeconds) seconds for all // subscriptions of the persistent-topic - ExpireAllMessages(TopicName, int64) error + ExpireAllMessages(utils.TopicName, int64) error // PeekMessages peeks messages from a topic subscription - PeekMessages(TopicName, string, int) ([]*Message, error) + PeekMessages(utils.TopicName, string, int) ([]*utils.Message, error) } type subscriptions struct { @@ -84,63 +85,63 @@ func (c *client) Subscriptions() Subscriptions { } } -func (s *subscriptions) Create(topic TopicName, sName string, messageID MessageID) error { +func (s *subscriptions) Create(topic utils.TopicName, sName string, messageID utils.MessageID) error { endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName)) return s.client.put(endpoint, messageID) } -func (s *subscriptions) Delete(topic TopicName, sName string) error { +func (s *subscriptions) Delete(topic utils.TopicName, sName string) error { endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName)) return s.client.delete(endpoint) } -func (s *subscriptions) List(topic TopicName) ([]string, error) { +func (s *subscriptions) List(topic utils.TopicName) ([]string, error) { endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), "subscriptions") var list []string return list, s.client.get(endpoint, &list) } -func (s *subscriptions) ResetCursorToMessageID(topic TopicName, sName string, id MessageID) error { +func (s *subscriptions) ResetCursorToMessageID(topic utils.TopicName, sName string, id utils.MessageID) error { endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "resetcursor") return s.client.post(endpoint, id) } -func (s *subscriptions) ResetCursorToTimestamp(topic TopicName, sName string, timestamp int64) error { +func (s *subscriptions) ResetCursorToTimestamp(topic utils.TopicName, sName string, timestamp int64) error { endpoint := s.client.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "resetcursor", strconv.FormatInt(timestamp, 10)) return s.client.post(endpoint, "") } -func (s *subscriptions) ClearBacklog(topic TopicName, sName string) error { +func (s *subscriptions) ClearBacklog(topic utils.TopicName, sName string) error { endpoint := s.client.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "skip_all") return s.client.post(endpoint, "") } -func (s *subscriptions) SkipMessages(topic TopicName, sName string, n int64) error { +func (s *subscriptions) SkipMessages(topic utils.TopicName, sName string, n int64) error { endpoint := s.client.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "skip", strconv.FormatInt(n, 10)) return s.client.post(endpoint, "") } -func (s *subscriptions) ExpireMessages(topic TopicName, sName string, expire int64) error { +func (s *subscriptions) ExpireMessages(topic utils.TopicName, sName string, expire int64) error { endpoint := s.client.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "expireMessages", strconv.FormatInt(expire, 10)) return s.client.post(endpoint, "") } -func (s *subscriptions) ExpireAllMessages(topic TopicName, expire int64) error { +func (s *subscriptions) ExpireAllMessages(topic utils.TopicName, expire int64) error { endpoint := s.client.endpoint( s.basePath, topic.GetRestPath(), "all_subscription", "expireMessages", strconv.FormatInt(expire, 10)) return s.client.post(endpoint, "") } -func (s *subscriptions) PeekMessages(topic TopicName, sName string, n int) ([]*Message, error) { - var msgs []*Message +func (s *subscriptions) PeekMessages(topic utils.TopicName, sName string, n int) ([]*utils.Message, error) { + var msgs []*utils.Message count := 1 for n > 0 { @@ -156,7 +157,7 @@ func (s *subscriptions) PeekMessages(topic TopicName, sName string, n int) ([]*M return msgs, nil } -func (s *subscriptions) peekNthMessage(topic TopicName, sName string, pos int) ([]*Message, error) { +func (s *subscriptions) peekNthMessage(topic utils.TopicName, sName string, pos int) ([]*utils.Message, error) { endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), "subscription", url.QueryEscape(sName), "position", strconv.Itoa(pos)) req, err := s.client.newRequest(http.MethodGet, endpoint) @@ -179,9 +180,9 @@ const ( PropertyPrefix = "X-Pulsar-PROPERTY-" ) -func handleResp(topic TopicName, resp *http.Response) ([]*Message, error) { +func handleResp(topic utils.TopicName, resp *http.Response) ([]*utils.Message, error) { msgID := resp.Header.Get("X-Pulsar-Message-ID") - ID, err := ParseMessageID(msgID) + ID, err := utils.ParseMessageID(msgID) if err != nil { return nil, err } @@ -212,18 +213,18 @@ func handleResp(topic TopicName, resp *http.Response) ([]*Message, error) { } } - return []*Message{NewMessage(topic.String(), *ID, payload, properties)}, nil + return []*utils.Message{utils.NewMessage(topic.String(), *ID, payload, properties)}, nil } -func getIndividualMsgsFromBatch(topic TopicName, msgID *MessageID, data []byte, - properties map[string]string) ([]*Message, error) { +func getIndividualMsgsFromBatch(topic utils.TopicName, msgID *utils.MessageID, data []byte, + properties map[string]string) ([]*utils.Message, error) { batchSize, err := strconv.Atoi(properties[BatchHeader]) if err != nil { return nil, nil } - msgs := make([]*Message, 0, batchSize) + msgs := make([]*utils.Message, 0, batchSize) // read all messages in batch buf32 := make([]byte, 4) @@ -242,7 +243,7 @@ func getIndividualMsgsFromBatch(topic TopicName, msgID *MessageID, data []byte, return nil, err } - singleMeta := new(SingleMessageMetadata) + singleMeta := new(utils.SingleMessageMetadata) if err := proto.Unmarshal(singleMetaBuf, singleMeta); err != nil { return nil, err } @@ -261,11 +262,11 @@ func getIndividualMsgsFromBatch(topic TopicName, msgID *MessageID, data []byte, return nil, err } - msgs = append(msgs, &Message{ - topic: topic.String(), - messageID: *msgID, - payload: singlePayload, - properties: properties, + msgs = append(msgs, &utils.Message{ + Topic: topic.String(), + MessageID: *msgID, + Payload: singlePayload, + Properties: properties, }) } diff --git a/pulsaradmin/pkg/pulsar/tenant.go b/pulsaradmin/pkg/pulsar/tenant.go index 78ce1a174f..038a282299 100644 --- a/pulsaradmin/pkg/pulsar/tenant.go +++ b/pulsaradmin/pkg/pulsar/tenant.go @@ -17,22 +17,26 @@ package pulsar +import ( + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" +) + // Tenants is admin interface for tenants management type Tenants interface { // Create a new tenant - Create(TenantData) error + Create(utils.TenantData) error // Delete an existing tenant Delete(string) error // Update the admins for a tenant - Update(TenantData) error + Update(utils.TenantData) error //List returns the list of tenants List() ([]string, error) // Get returns the config of the tenant. - Get(string) (TenantData, error) + Get(string) (utils.TenantData, error) } type tenants struct { @@ -48,7 +52,7 @@ func (c *client) Tenants() Tenants { } } -func (c *tenants) Create(data TenantData) error { +func (c *tenants) Create(data utils.TenantData) error { endpoint := c.client.endpoint(c.basePath, data.Name) return c.client.put(endpoint, &data) } @@ -58,7 +62,7 @@ func (c *tenants) Delete(name string) error { return c.client.delete(endpoint) } -func (c *tenants) Update(data TenantData) error { +func (c *tenants) Update(data utils.TenantData) error { endpoint := c.client.endpoint(c.basePath, data.Name) return c.client.post(endpoint, &data) } @@ -70,8 +74,8 @@ func (c *tenants) List() ([]string, error) { return tenantList, err } -func (c *tenants) Get(name string) (TenantData, error) { - var data TenantData +func (c *tenants) Get(name string) (utils.TenantData, error) { + var data utils.TenantData endpoint := c.client.endpoint(c.basePath, name) err := c.client.get(endpoint, &data) return data, err diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 9897256780..e11b1843fe 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -20,81 +20,84 @@ package pulsar import ( "fmt" "strconv" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) // Topics is admin interface for topics management type Topics interface { // Create a topic - Create(TopicName, int) error + Create(utils.TopicName, int) error // Delete a topic - Delete(TopicName, bool, bool) error + Delete(utils.TopicName, bool, bool) error // Update number of partitions of a non-global partitioned topic // It requires partitioned-topic to be already exist and number of new partitions must be greater than existing // number of partitions. Decrementing number of partitions requires deletion of topic which is not supported. - Update(TopicName, int) error + Update(utils.TopicName, int) error // GetMetadata returns metadata of a partitioned topic - GetMetadata(TopicName) (PartitionedTopicMetadata, error) + GetMetadata(utils.TopicName) (utils.PartitionedTopicMetadata, error) // List returns the list of topics under a namespace - List(NameSpaceName) ([]string, []string, error) + List(utils.NameSpaceName) ([]string, []string, error) // GetInternalInfo returns the internal metadata info for the topic - GetInternalInfo(TopicName) (ManagedLedgerInfo, error) + GetInternalInfo(utils.TopicName) (utils.ManagedLedgerInfo, error) // GetPermissions returns permissions on a topic // Retrieve the effective permissions for a topic. These permissions are defined by the permissions set at the // namespace level combined (union) with any eventual specific permission set on the topic. - GetPermissions(TopicName) (map[string][]AuthAction, error) + GetPermissions(utils.TopicName) (map[string][]common.AuthAction, error) // GrantPermission grants a new permission to a client role on a single topic - GrantPermission(TopicName, string, []AuthAction) error + GrantPermission(utils.TopicName, string, []common.AuthAction) error // RevokePermission revokes permissions to a client role on a single topic. If the permission // was not set at the topic level, but rather at the namespace level, this operation will // return an error (HTTP status code 412). - RevokePermission(TopicName, string) error + RevokePermission(utils.TopicName, string) error // Lookup a topic returns the broker URL that serves the topic - Lookup(TopicName) (LookupData, error) + Lookup(utils.TopicName) (utils.LookupData, error) // GetBundleRange returns a bundle range of a topic - GetBundleRange(TopicName) (string, error) + GetBundleRange(utils.TopicName) (string, error) // GetLastMessageID returns the last commit message Id of a topic - GetLastMessageID(TopicName) (MessageID, error) + GetLastMessageID(utils.TopicName) (utils.MessageID, error) // GetStats returns the stats for the topic // All the rates are computed over a 1 minute window and are relative the last completed 1 minute period - GetStats(TopicName) (TopicStats, error) + GetStats(utils.TopicName) (utils.TopicStats, error) // GetInternalStats returns the internal stats for the topic. - GetInternalStats(TopicName) (PersistentTopicInternalStats, error) + GetInternalStats(utils.TopicName) (utils.PersistentTopicInternalStats, error) // GetPartitionedStats returns the stats for the partitioned topic // All the rates are computed over a 1 minute window and are relative the last completed 1 minute period - GetPartitionedStats(TopicName, bool) (PartitionedTopicStats, error) + GetPartitionedStats(utils.TopicName, bool) (utils.PartitionedTopicStats, error) // Terminate the topic and prevent any more messages being published on it - Terminate(TopicName) (MessageID, error) + Terminate(utils.TopicName) (utils.MessageID, error) // Offload triggers offloading messages in topic to longterm storage - Offload(TopicName, MessageID) error + Offload(utils.TopicName, utils.MessageID) error // OffloadStatus checks the status of an ongoing offloading operation for a topic - OffloadStatus(TopicName) (OffloadProcessStatus, error) + OffloadStatus(utils.TopicName) (utils.OffloadProcessStatus, error) // Unload a topic - Unload(TopicName) error + Unload(utils.TopicName) error // Compact triggers compaction to run for a topic. A single topic can only have one instance of compaction // running at any time. Any attempt to trigger another will be met with a ConflictException. - Compact(TopicName) error + Compact(utils.TopicName) error // CompactStatus checks the status of an ongoing compaction for a topic - CompactStatus(TopicName) (LongRunningProcessStatus, error) + CompactStatus(utils.TopicName) (utils.LongRunningProcessStatus, error) } type topics struct { @@ -116,7 +119,7 @@ func (c *client) Topics() Topics { } } -func (t *topics) Create(topic TopicName, partitions int) error { +func (t *topics) Create(topic utils.TopicName, partitions int) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") if partitions == 0 { endpoint = t.client.endpoint(t.basePath, topic.GetRestPath()) @@ -124,7 +127,7 @@ func (t *topics) Create(topic TopicName, partitions int) error { return t.client.put(endpoint, partitions) } -func (t *topics) Delete(topic TopicName, force bool, nonPartitioned bool) error { +func (t *topics) Delete(topic utils.TopicName, force bool, nonPartitioned bool) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") if nonPartitioned { endpoint = t.client.endpoint(t.basePath, topic.GetRestPath()) @@ -135,19 +138,19 @@ func (t *topics) Delete(topic TopicName, force bool, nonPartitioned bool) error return t.client.deleteWithQueryParams(endpoint, nil, params) } -func (t *topics) Update(topic TopicName, partitions int) error { +func (t *topics) Update(topic utils.TopicName, partitions int) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") return t.client.post(endpoint, partitions) } -func (t *topics) GetMetadata(topic TopicName) (PartitionedTopicMetadata, error) { +func (t *topics) GetMetadata(topic utils.TopicName) (utils.PartitionedTopicMetadata, error) { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") - var partitionedMeta PartitionedTopicMetadata + var partitionedMeta utils.PartitionedTopicMetadata err := t.client.get(endpoint, &partitionedMeta) return partitionedMeta, err } -func (t *topics) List(namespace NameSpaceName) ([]string, []string, error) { +func (t *topics) List(namespace utils.NameSpaceName) ([]string, []string, error) { var partitionedTopics, nonPartitionedTopics []string partitionedTopicsChan := make(chan []string) nonPartitionedTopicsChan := make(chan []string) @@ -191,21 +194,21 @@ func (t *topics) getTopics(endpoint string, out chan<- []string, err chan<- erro out <- topics } -func (t *topics) GetInternalInfo(topic TopicName) (ManagedLedgerInfo, error) { +func (t *topics) GetInternalInfo(topic utils.TopicName) (utils.ManagedLedgerInfo, error) { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "internal-info") - var info ManagedLedgerInfo + var info utils.ManagedLedgerInfo err := t.client.get(endpoint, &info) return info, err } -func (t *topics) GetPermissions(topic TopicName) (map[string][]AuthAction, error) { - var permissions map[string][]AuthAction +func (t *topics) GetPermissions(topic utils.TopicName) (map[string][]common.AuthAction, error) { + var permissions map[string][]common.AuthAction endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions") err := t.client.get(endpoint, &permissions) return permissions, err } -func (t *topics) GrantPermission(topic TopicName, role string, action []AuthAction) error { +func (t *topics) GrantPermission(topic utils.TopicName, role string, action []common.AuthAction) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions", role) s := []string{} for _, v := range action { @@ -214,47 +217,47 @@ func (t *topics) GrantPermission(topic TopicName, role string, action []AuthActi return t.client.post(endpoint, s) } -func (t *topics) RevokePermission(topic TopicName, role string) error { +func (t *topics) RevokePermission(topic utils.TopicName, role string) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions", role) return t.client.delete(endpoint) } -func (t *topics) Lookup(topic TopicName) (LookupData, error) { - var lookup LookupData +func (t *topics) Lookup(topic utils.TopicName) (utils.LookupData, error) { + var lookup utils.LookupData endpoint := fmt.Sprintf("%s/%s", t.lookupPath, topic.GetRestPath()) err := t.client.get(endpoint, &lookup) return lookup, err } -func (t *topics) GetBundleRange(topic TopicName) (string, error) { +func (t *topics) GetBundleRange(topic utils.TopicName) (string, error) { endpoint := fmt.Sprintf("%s/%s/%s", t.lookupPath, topic.GetRestPath(), "bundle") data, err := t.client.getWithQueryParams(endpoint, nil, nil, false) return string(data), err } -func (t *topics) GetLastMessageID(topic TopicName) (MessageID, error) { - var messageID MessageID +func (t *topics) GetLastMessageID(topic utils.TopicName) (utils.MessageID, error) { + var messageID utils.MessageID endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "lastMessageId") err := t.client.get(endpoint, &messageID) return messageID, err } -func (t *topics) GetStats(topic TopicName) (TopicStats, error) { - var stats TopicStats +func (t *topics) GetStats(topic utils.TopicName) (utils.TopicStats, error) { + var stats utils.TopicStats endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "stats") err := t.client.get(endpoint, &stats) return stats, err } -func (t *topics) GetInternalStats(topic TopicName) (PersistentTopicInternalStats, error) { - var stats PersistentTopicInternalStats +func (t *topics) GetInternalStats(topic utils.TopicName) (utils.PersistentTopicInternalStats, error) { + var stats utils.PersistentTopicInternalStats endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "internalStats") err := t.client.get(endpoint, &stats) return stats, err } -func (t *topics) GetPartitionedStats(topic TopicName, perPartition bool) (PartitionedTopicStats, error) { - var stats PartitionedTopicStats +func (t *topics) GetPartitionedStats(topic utils.TopicName, perPartition bool) (utils.PartitionedTopicStats, error) { + var stats utils.PartitionedTopicStats endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitioned-stats") params := map[string]string{ "perPartition": strconv.FormatBool(perPartition), @@ -263,38 +266,38 @@ func (t *topics) GetPartitionedStats(topic TopicName, perPartition bool) (Partit return stats, err } -func (t *topics) Terminate(topic TopicName) (MessageID, error) { +func (t *topics) Terminate(topic utils.TopicName) (utils.MessageID, error) { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "terminate") - var messageID MessageID + var messageID utils.MessageID err := t.client.postWithObj(endpoint, "", &messageID) return messageID, err } -func (t *topics) Offload(topic TopicName, messageID MessageID) error { +func (t *topics) Offload(topic utils.TopicName, messageID utils.MessageID) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "offload") return t.client.put(endpoint, messageID) } -func (t *topics) OffloadStatus(topic TopicName) (OffloadProcessStatus, error) { +func (t *topics) OffloadStatus(topic utils.TopicName) (utils.OffloadProcessStatus, error) { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "offload") - var status OffloadProcessStatus + var status utils.OffloadProcessStatus err := t.client.get(endpoint, &status) return status, err } -func (t *topics) Unload(topic TopicName) error { +func (t *topics) Unload(topic utils.TopicName) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "unload") return t.client.put(endpoint, "") } -func (t *topics) Compact(topic TopicName) error { +func (t *topics) Compact(topic utils.TopicName) error { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "compaction") return t.client.put(endpoint, "") } -func (t *topics) CompactStatus(topic TopicName) (LongRunningProcessStatus, error) { +func (t *topics) CompactStatus(topic utils.TopicName) (utils.LongRunningProcessStatus, error) { endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "compaction") - var status LongRunningProcessStatus + var status utils.LongRunningProcessStatus err := t.client.get(endpoint, &status) return status, err } diff --git a/pulsaradmin/pkg/pulsar/allocator_stats.go b/pulsaradmin/pkg/pulsar/utils/allocator_stats.go similarity index 99% rename from pulsaradmin/pkg/pulsar/allocator_stats.go rename to pulsaradmin/pkg/pulsar/utils/allocator_stats.go index 54c2d6a3cb..4c2a7fb521 100644 --- a/pulsaradmin/pkg/pulsar/allocator_stats.go +++ b/pulsaradmin/pkg/pulsar/utils/allocator_stats.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type AllocatorStats struct { NumDirectArenas int `json:"numDirectArenas"` diff --git a/pulsaradmin/pkg/pulsar/backlog_quota.go b/pulsaradmin/pkg/pulsar/utils/backlog_quota.go similarity index 98% rename from pulsaradmin/pkg/pulsar/backlog_quota.go rename to pulsaradmin/pkg/pulsar/utils/backlog_quota.go index c3ece4cf38..17139e7aad 100644 --- a/pulsaradmin/pkg/pulsar/backlog_quota.go +++ b/pulsaradmin/pkg/pulsar/utils/backlog_quota.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type BacklogQuota struct { Limit int64 `json:"limit"` diff --git a/pulsaradmin/pkg/pulsar/broker_ns_isolation_data.go b/pulsaradmin/pkg/pulsar/utils/broker_ns_isolation_data.go similarity index 98% rename from pulsaradmin/pkg/pulsar/broker_ns_isolation_data.go rename to pulsaradmin/pkg/pulsar/utils/broker_ns_isolation_data.go index 29b9ef55c5..1381647b6b 100644 --- a/pulsaradmin/pkg/pulsar/broker_ns_isolation_data.go +++ b/pulsaradmin/pkg/pulsar/utils/broker_ns_isolation_data.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type BrokerNamespaceIsolationData struct { BrokerName string `json:"brokerName"` diff --git a/pulsaradmin/pkg/pulsar/bundles_data.go b/pulsaradmin/pkg/pulsar/utils/bundles_data.go similarity index 98% rename from pulsaradmin/pkg/pulsar/bundles_data.go rename to pulsaradmin/pkg/pulsar/utils/bundles_data.go index 5da16ae44e..2bba7ae493 100644 --- a/pulsaradmin/pkg/pulsar/bundles_data.go +++ b/pulsaradmin/pkg/pulsar/utils/bundles_data.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type BundlesData struct { Boundaries []string `json:"boundaries"` diff --git a/pulsaradmin/pkg/pulsar/connector_definition.go b/pulsaradmin/pkg/pulsar/utils/connector_definition.go similarity index 98% rename from pulsaradmin/pkg/pulsar/connector_definition.go rename to pulsaradmin/pkg/pulsar/utils/connector_definition.go index 71b3aad290..72b12893cd 100644 --- a/pulsaradmin/pkg/pulsar/connector_definition.go +++ b/pulsaradmin/pkg/pulsar/utils/connector_definition.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils // Basic information about a Pulsar connector type ConnectorDefinition struct { diff --git a/pulsaradmin/pkg/pulsar/consumer_config.go b/pulsaradmin/pkg/pulsar/utils/consumer_config.go similarity index 98% rename from pulsaradmin/pkg/pulsar/consumer_config.go rename to pulsaradmin/pkg/pulsar/utils/consumer_config.go index 5cf84971c2..8d4aaf482d 100644 --- a/pulsaradmin/pkg/pulsar/consumer_config.go +++ b/pulsaradmin/pkg/pulsar/utils/consumer_config.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type ConsumerConfig struct { SchemaType string diff --git a/pulsaradmin/pkg/pulsar/data.go b/pulsaradmin/pkg/pulsar/utils/data.go similarity index 99% rename from pulsaradmin/pkg/pulsar/data.go rename to pulsaradmin/pkg/pulsar/utils/data.go index 003ef149a3..ab0ae08026 100644 --- a/pulsaradmin/pkg/pulsar/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils // ClusterData information on a cluster type ClusterData struct { diff --git a/pulsaradmin/pkg/pulsar/dispatch_rate.go b/pulsaradmin/pkg/pulsar/utils/dispatch_rate.go similarity index 99% rename from pulsaradmin/pkg/pulsar/dispatch_rate.go rename to pulsaradmin/pkg/pulsar/utils/dispatch_rate.go index 7ccd3d758a..9c6ccce88c 100644 --- a/pulsaradmin/pkg/pulsar/dispatch_rate.go +++ b/pulsaradmin/pkg/pulsar/utils/dispatch_rate.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type DispatchRate struct { DispatchThrottlingRateInMsg int `json:"dispatchThrottlingRateInMsg"` diff --git a/pulsaradmin/pkg/pulsar/function_confg.go b/pulsaradmin/pkg/pulsar/utils/function_confg.go similarity index 99% rename from pulsaradmin/pkg/pulsar/function_confg.go rename to pulsaradmin/pkg/pulsar/utils/function_confg.go index 0590ec1194..8b312f28f9 100644 --- a/pulsaradmin/pkg/pulsar/function_confg.go +++ b/pulsaradmin/pkg/pulsar/utils/function_confg.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils const ( JavaRuntime = "JAVA" diff --git a/pulsaradmin/pkg/pulsar/function_state.go b/pulsaradmin/pkg/pulsar/utils/function_state.go similarity index 98% rename from pulsaradmin/pkg/pulsar/function_state.go rename to pulsaradmin/pkg/pulsar/utils/function_state.go index a740e20e2e..63fa15057c 100644 --- a/pulsaradmin/pkg/pulsar/function_state.go +++ b/pulsaradmin/pkg/pulsar/utils/function_state.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type FunctionState struct { Key string `json:"key"` diff --git a/pulsaradmin/pkg/pulsar/function_status.go b/pulsaradmin/pkg/pulsar/utils/function_status.go similarity index 99% rename from pulsaradmin/pkg/pulsar/function_status.go rename to pulsaradmin/pkg/pulsar/utils/function_status.go index b560eec620..360e61b077 100644 --- a/pulsaradmin/pkg/pulsar/function_status.go +++ b/pulsaradmin/pkg/pulsar/utils/function_status.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type FunctionStatus struct { NumInstances int `json:"numInstances"` diff --git a/pulsaradmin/pkg/pulsar/functions_stats.go b/pulsaradmin/pkg/pulsar/utils/functions_stats.go similarity index 99% rename from pulsaradmin/pkg/pulsar/functions_stats.go rename to pulsaradmin/pkg/pulsar/utils/functions_stats.go index 3e6ecc4efe..82761afe51 100644 --- a/pulsaradmin/pkg/pulsar/functions_stats.go +++ b/pulsaradmin/pkg/pulsar/utils/functions_stats.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type FunctionStats struct { // Overall total number of records function received from source diff --git a/pulsaradmin/pkg/pulsar/internal_configuration_data.go b/pulsaradmin/pkg/pulsar/utils/internal_configuration_data.go similarity index 98% rename from pulsaradmin/pkg/pulsar/internal_configuration_data.go rename to pulsaradmin/pkg/pulsar/utils/internal_configuration_data.go index 9485164e2e..75cad0e245 100644 --- a/pulsaradmin/pkg/pulsar/internal_configuration_data.go +++ b/pulsaradmin/pkg/pulsar/utils/internal_configuration_data.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type InternalConfigurationData struct { ZookeeperServers string `json:"zookeeperServers"` diff --git a/pulsaradmin/pkg/pulsar/load_manage_report.go b/pulsaradmin/pkg/pulsar/utils/load_manage_report.go similarity index 99% rename from pulsaradmin/pkg/pulsar/load_manage_report.go rename to pulsaradmin/pkg/pulsar/utils/load_manage_report.go index 518ee67169..5196da9b55 100644 --- a/pulsaradmin/pkg/pulsar/load_manage_report.go +++ b/pulsaradmin/pkg/pulsar/utils/load_manage_report.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import ( "math" diff --git a/pulsaradmin/pkg/pulsar/long_running_process_status.go b/pulsaradmin/pkg/pulsar/utils/long_running_process_status.go similarity index 98% rename from pulsaradmin/pkg/pulsar/long_running_process_status.go rename to pulsaradmin/pkg/pulsar/utils/long_running_process_status.go index 177f14bd5b..c61919ecc8 100644 --- a/pulsaradmin/pkg/pulsar/long_running_process_status.go +++ b/pulsaradmin/pkg/pulsar/utils/long_running_process_status.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type Status string diff --git a/pulsaradmin/pkg/pulsar/message.go b/pulsaradmin/pkg/pulsar/utils/message.go similarity index 92% rename from pulsaradmin/pkg/pulsar/message.go rename to pulsaradmin/pkg/pulsar/utils/message.go index 0e364344c6..d6207a2c1a 100644 --- a/pulsaradmin/pkg/pulsar/message.go +++ b/pulsaradmin/pkg/pulsar/utils/message.go @@ -15,36 +15,36 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import "github.com/golang/protobuf/proto" type Message struct { - messageID MessageID - payload []byte - topic string - properties map[string]string + MessageID MessageID + Payload []byte + Topic string + Properties map[string]string } func NewMessage(topic string, id MessageID, payload []byte, properties map[string]string) *Message { return &Message{ - messageID: id, - payload: payload, - topic: topic, - properties: properties, + MessageID: id, + Payload: payload, + Topic: topic, + Properties: properties, } } func (m *Message) GetMessageID() MessageID { - return m.messageID + return m.MessageID } func (m *Message) GetProperties() map[string]string { - return m.properties + return m.Properties } func (m *Message) GetPayload() []byte { - return m.payload + return m.Payload } // nolint diff --git a/pulsaradmin/pkg/pulsar/message_id.go b/pulsaradmin/pkg/pulsar/utils/message_id.go similarity index 99% rename from pulsaradmin/pkg/pulsar/message_id.go rename to pulsaradmin/pkg/pulsar/utils/message_id.go index 898f15b949..34734ac428 100644 --- a/pulsaradmin/pkg/pulsar/message_id.go +++ b/pulsaradmin/pkg/pulsar/utils/message_id.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import ( "strconv" diff --git a/pulsaradmin/pkg/pulsar/message_id_test.go b/pulsaradmin/pkg/pulsar/utils/message_id_test.go similarity index 99% rename from pulsaradmin/pkg/pulsar/message_id_test.go rename to pulsaradmin/pkg/pulsar/utils/message_id_test.go index 46c76b7423..320b51ae10 100644 --- a/pulsaradmin/pkg/pulsar/message_id_test.go +++ b/pulsaradmin/pkg/pulsar/utils/message_id_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import ( "testing" diff --git a/pulsaradmin/pkg/pulsar/metrics.go b/pulsaradmin/pkg/pulsar/utils/metrics.go similarity index 98% rename from pulsaradmin/pkg/pulsar/metrics.go rename to pulsaradmin/pkg/pulsar/utils/metrics.go index 042b8a983e..ca1919423d 100644 --- a/pulsaradmin/pkg/pulsar/metrics.go +++ b/pulsaradmin/pkg/pulsar/utils/metrics.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type Metrics struct { Metrics map[string]interface{} `json:"metrics"` diff --git a/pulsaradmin/pkg/pulsar/namespace_name.go b/pulsaradmin/pkg/pulsar/utils/namespace_name.go similarity index 99% rename from pulsaradmin/pkg/pulsar/namespace_name.go rename to pulsaradmin/pkg/pulsar/utils/namespace_name.go index ac57214611..fbf36acd7f 100644 --- a/pulsaradmin/pkg/pulsar/namespace_name.go +++ b/pulsaradmin/pkg/pulsar/utils/namespace_name.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import ( "fmt" diff --git a/pulsaradmin/pkg/pulsar/namespace_name_test.go b/pulsaradmin/pkg/pulsar/utils/namespace_name_test.go similarity index 99% rename from pulsaradmin/pkg/pulsar/namespace_name_test.go rename to pulsaradmin/pkg/pulsar/utils/namespace_name_test.go index 4b5294afee..bc12070e9a 100644 --- a/pulsaradmin/pkg/pulsar/namespace_name_test.go +++ b/pulsaradmin/pkg/pulsar/utils/namespace_name_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import ( "testing" diff --git a/pulsaradmin/pkg/pulsar/ns_isolation_data.go b/pulsaradmin/pkg/pulsar/utils/ns_isolation_data.go similarity index 99% rename from pulsaradmin/pkg/pulsar/ns_isolation_data.go rename to pulsaradmin/pkg/pulsar/utils/ns_isolation_data.go index bb87a1a5e2..4589eb30c3 100644 --- a/pulsaradmin/pkg/pulsar/ns_isolation_data.go +++ b/pulsaradmin/pkg/pulsar/utils/ns_isolation_data.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import ( "github.com/pkg/errors" diff --git a/pulsaradmin/pkg/pulsar/ns_ownership_status.go b/pulsaradmin/pkg/pulsar/utils/ns_ownership_status.go similarity index 98% rename from pulsaradmin/pkg/pulsar/ns_ownership_status.go rename to pulsaradmin/pkg/pulsar/utils/ns_ownership_status.go index 6cf37b0a0e..bb63b704f8 100644 --- a/pulsaradmin/pkg/pulsar/ns_ownership_status.go +++ b/pulsaradmin/pkg/pulsar/utils/ns_ownership_status.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type NamespaceOwnershipStatus struct { BrokerAssignment BrokerAssignment `json:"broker_assignment"` diff --git a/pulsaradmin/pkg/pulsar/persistence_policies.go b/pulsaradmin/pkg/pulsar/utils/persistence_policies.go similarity index 99% rename from pulsaradmin/pkg/pulsar/persistence_policies.go rename to pulsaradmin/pkg/pulsar/utils/persistence_policies.go index 3c0a2229ef..d4c8bdb04e 100644 --- a/pulsaradmin/pkg/pulsar/persistence_policies.go +++ b/pulsaradmin/pkg/pulsar/utils/persistence_policies.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type PersistencePolicies struct { BookkeeperEnsemble int `json:"bookkeeperEnsemble"` diff --git a/pulsaradmin/pkg/pulsar/policies.go b/pulsaradmin/pkg/pulsar/utils/policies.go similarity index 95% rename from pulsaradmin/pkg/pulsar/policies.go rename to pulsaradmin/pkg/pulsar/utils/policies.go index 392f416c16..3f6f897821 100644 --- a/pulsaradmin/pkg/pulsar/policies.go +++ b/pulsaradmin/pkg/pulsar/utils/policies.go @@ -15,7 +15,11 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils + +import ( + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" +) const ( FirstBoundary string = "0x00000000" @@ -46,13 +50,13 @@ type Policies struct { ReplicatorDispatchRate map[string]DispatchRate `json:"replicatorDispatchRate"` ClusterSubscribeRate map[string]SubscribeRate `json:"clusterSubscribeRate"` SchemaCompatibilityStrategy SchemaCompatibilityStrategy `json:"schema_auto_update_compatibility_strategy"` - AuthPolicies AuthPolicies `json:"auth_policies"` + AuthPolicies common.AuthPolicies `json:"auth_policies"` SubscriptionAuthMode SubscriptionAuthMode `json:"subscription_auth_mode"` } func NewDefaultPolicies() *Policies { return &Policies{ - AuthPolicies: *NewAuthPolicies(), + AuthPolicies: *common.NewAuthPolicies(), ReplicationClusters: make([]string, 0, 10), BacklogQuotaMap: make(map[BacklogQuotaType]BacklogQuota), TopicDispatchRate: make(map[string]DispatchRate), diff --git a/pulsaradmin/pkg/pulsar/resource_quota.go b/pulsaradmin/pkg/pulsar/utils/resource_quota.go similarity index 99% rename from pulsaradmin/pkg/pulsar/resource_quota.go rename to pulsaradmin/pkg/pulsar/utils/resource_quota.go index 40bd585990..d9ef4f74db 100644 --- a/pulsaradmin/pkg/pulsar/resource_quota.go +++ b/pulsaradmin/pkg/pulsar/utils/resource_quota.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type ResourceQuota struct { // messages published per second diff --git a/pulsaradmin/pkg/pulsar/resources.go b/pulsaradmin/pkg/pulsar/utils/resources.go similarity index 98% rename from pulsaradmin/pkg/pulsar/resources.go rename to pulsaradmin/pkg/pulsar/utils/resources.go index bfe03375a8..f5da7b87a1 100644 --- a/pulsaradmin/pkg/pulsar/resources.go +++ b/pulsaradmin/pkg/pulsar/utils/resources.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type Resources struct { CPU float64 `json:"cpu"` diff --git a/pulsaradmin/pkg/pulsar/retention_policies.go b/pulsaradmin/pkg/pulsar/utils/retention_policies.go similarity index 98% rename from pulsaradmin/pkg/pulsar/retention_policies.go rename to pulsaradmin/pkg/pulsar/utils/retention_policies.go index 23e3d7c736..55bf915ed2 100644 --- a/pulsaradmin/pkg/pulsar/retention_policies.go +++ b/pulsaradmin/pkg/pulsar/utils/retention_policies.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type RetentionPolicies struct { RetentionTimeInMinutes int `json:"retentionTimeInMinutes"` diff --git a/pulsaradmin/pkg/pulsar/schema_strategy.go b/pulsaradmin/pkg/pulsar/utils/schema_strategy.go similarity index 99% rename from pulsaradmin/pkg/pulsar/schema_strategy.go rename to pulsaradmin/pkg/pulsar/utils/schema_strategy.go index 309fd07334..176f0e0a90 100644 --- a/pulsaradmin/pkg/pulsar/schema_strategy.go +++ b/pulsaradmin/pkg/pulsar/utils/schema_strategy.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import "github.com/pkg/errors" diff --git a/pulsaradmin/pkg/pulsar/schema_util.go b/pulsaradmin/pkg/pulsar/utils/schema_util.go similarity index 91% rename from pulsaradmin/pkg/pulsar/schema_util.go rename to pulsaradmin/pkg/pulsar/utils/schema_util.go index 08c3dab875..08aaf54ac6 100644 --- a/pulsaradmin/pkg/pulsar/schema_util.go +++ b/pulsaradmin/pkg/pulsar/utils/schema_util.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type SchemaInfo struct { Name string `json:"name"` @@ -44,7 +44,7 @@ type GetSchemaResponse struct { Properties map[string]string `json:"properties"` } -func convertGetSchemaResponseToSchemaInfo(tn *TopicName, response GetSchemaResponse) *SchemaInfo { +func ConvertGetSchemaResponseToSchemaInfo(tn *TopicName, response GetSchemaResponse) *SchemaInfo { info := new(SchemaInfo) schema := make([]byte, 0, 10) if response.Type == "KEY_VALUE" { @@ -61,9 +61,9 @@ func convertGetSchemaResponseToSchemaInfo(tn *TopicName, response GetSchemaRespo return info } -func convertGetSchemaResponseToSchemaInfoWithVersion(tn *TopicName, response GetSchemaResponse) *SchemaInfoWithVersion { +func ConvertGetSchemaResponseToSchemaInfoWithVersion(tn *TopicName, response GetSchemaResponse) *SchemaInfoWithVersion { info := new(SchemaInfoWithVersion) - info.SchemaInfo = convertGetSchemaResponseToSchemaInfo(tn, response) + info.SchemaInfo = ConvertGetSchemaResponseToSchemaInfo(tn, response) info.Version = response.Version return info } diff --git a/pulsaradmin/pkg/pulsar/sinkConfig.go b/pulsaradmin/pkg/pulsar/utils/sink_config.go similarity index 99% rename from pulsaradmin/pkg/pulsar/sinkConfig.go rename to pulsaradmin/pkg/pulsar/utils/sink_config.go index 81fee99266..efa13d31c5 100644 --- a/pulsaradmin/pkg/pulsar/sinkConfig.go +++ b/pulsaradmin/pkg/pulsar/utils/sink_config.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type SinkConfig struct { TopicsPattern *string `json:"topicsPattern" yaml:"topicsPattern"` diff --git a/pulsaradmin/pkg/pulsar/sink_status.go b/pulsaradmin/pkg/pulsar/utils/sink_status.go similarity index 99% rename from pulsaradmin/pkg/pulsar/sink_status.go rename to pulsaradmin/pkg/pulsar/utils/sink_status.go index 5629914ecc..6cdb091fac 100644 --- a/pulsaradmin/pkg/pulsar/sink_status.go +++ b/pulsaradmin/pkg/pulsar/utils/sink_status.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type SinkStatus struct { // The total number of sink instances that ought to be running diff --git a/pulsaradmin/pkg/pulsar/source_config.go b/pulsaradmin/pkg/pulsar/utils/source_config.go similarity index 99% rename from pulsaradmin/pkg/pulsar/source_config.go rename to pulsaradmin/pkg/pulsar/utils/source_config.go index debbc39c4b..57b168b959 100644 --- a/pulsaradmin/pkg/pulsar/source_config.go +++ b/pulsaradmin/pkg/pulsar/utils/source_config.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type SourceConfig struct { Tenant string `json:"tenant" yaml:"tenant"` diff --git a/pulsaradmin/pkg/pulsar/source_status.go b/pulsaradmin/pkg/pulsar/utils/source_status.go similarity index 99% rename from pulsaradmin/pkg/pulsar/source_status.go rename to pulsaradmin/pkg/pulsar/utils/source_status.go index 1e147980ef..71df5a4fa8 100644 --- a/pulsaradmin/pkg/pulsar/source_status.go +++ b/pulsaradmin/pkg/pulsar/utils/source_status.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type SourceStatus struct { NumInstances int `json:"numInstances"` diff --git a/pulsaradmin/pkg/pulsar/subscription_auth_mode.go b/pulsaradmin/pkg/pulsar/utils/subscription_auth_mode.go similarity index 98% rename from pulsaradmin/pkg/pulsar/subscription_auth_mode.go rename to pulsaradmin/pkg/pulsar/utils/subscription_auth_mode.go index f55ab1997a..795b6d0aea 100644 --- a/pulsaradmin/pkg/pulsar/subscription_auth_mode.go +++ b/pulsaradmin/pkg/pulsar/utils/subscription_auth_mode.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import "github.com/pkg/errors" diff --git a/pulsaradmin/pkg/pulsar/topic_domain.go b/pulsaradmin/pkg/pulsar/utils/topic_domain.go similarity index 98% rename from pulsaradmin/pkg/pulsar/topic_domain.go rename to pulsaradmin/pkg/pulsar/utils/topic_domain.go index 9210a507e5..98c59a9cec 100644 --- a/pulsaradmin/pkg/pulsar/topic_domain.go +++ b/pulsaradmin/pkg/pulsar/utils/topic_domain.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import "github.com/pkg/errors" diff --git a/pulsaradmin/pkg/pulsar/topic_name.go b/pulsaradmin/pkg/pulsar/utils/topic_name.go similarity index 87% rename from pulsaradmin/pkg/pulsar/topic_name.go rename to pulsaradmin/pkg/pulsar/utils/topic_name.go index f18db1d060..52c44aae09 100644 --- a/pulsaradmin/pkg/pulsar/topic_name.go +++ b/pulsaradmin/pkg/pulsar/utils/topic_name.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import ( "fmt" @@ -45,7 +45,7 @@ type TopicName struct { // The topic name can be in two different forms, one is fully qualified topic name, // the other one is short topic name func GetTopicName(completeName string) (*TopicName, error) { - var topicname TopicName + var topicName TopicName // The short topic name can be: // - // - // @@ -71,31 +71,31 @@ func GetTopicName(completeName string) (*TopicName, error) { if err != nil { return nil, err } - topicname.domain = domain + topicName.domain = domain rest := parts[1] parts = strings.SplitN(rest, "/", 3) if len(parts) == 3 { - topicname.tenant = parts[0] - topicname.namespace = parts[1] - topicname.topic = parts[2] - topicname.partitionIndex = getPartitionIndex(completeName) + topicName.tenant = parts[0] + topicName.namespace = parts[1] + topicName.topic = parts[2] + topicName.partitionIndex = getPartitionIndex(completeName) } else { return nil, errors.Errorf("invalid topic name '%s', it should be in the format of "+ "//", rest) } - if topicname.topic == "" { + if topicName.topic == "" { return nil, errors.New("topic name can not be empty") } - n, err := GetNameSpaceName(topicname.tenant, topicname.namespace) + n, err := GetNameSpaceName(topicName.tenant, topicName.namespace) if err != nil { return nil, err } - topicname.namespaceName = n + topicName.namespaceName = n - return &topicname, nil + return &topicName, nil } func (t *TopicName) String() string { @@ -106,6 +106,14 @@ func (t *TopicName) GetDomain() TopicDomain { return t.domain } +func (t *TopicName) GetTenant() string { + return t.tenant +} + +func (t *TopicName) GetNamespace() string { + return t.namespace +} + func (t *TopicName) IsPersistent() bool { return t.domain == persistent } diff --git a/pulsaradmin/pkg/pulsar/topic_name_test.go b/pulsaradmin/pkg/pulsar/utils/topic_name_test.go similarity index 99% rename from pulsaradmin/pkg/pulsar/topic_name_test.go rename to pulsaradmin/pkg/pulsar/utils/topic_name_test.go index 55fc5090cd..8eba39ce64 100644 --- a/pulsaradmin/pkg/pulsar/topic_name_test.go +++ b/pulsaradmin/pkg/pulsar/utils/topic_name_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import ( "net/url" diff --git a/pulsaradmin/pkg/pulsar/topics_stats_stream.go b/pulsaradmin/pkg/pulsar/utils/topics_stats_stream.go similarity index 98% rename from pulsaradmin/pkg/pulsar/topics_stats_stream.go rename to pulsaradmin/pkg/pulsar/utils/topics_stats_stream.go index 7aa701e86b..1af615a2d2 100644 --- a/pulsaradmin/pkg/pulsar/topics_stats_stream.go +++ b/pulsaradmin/pkg/pulsar/utils/topics_stats_stream.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils //var TopicsMap map[string]map[string]map[string]TopicStats diff --git a/pulsaradmin/pkg/pulsar/update_options.go b/pulsaradmin/pkg/pulsar/utils/update_options.go similarity index 98% rename from pulsaradmin/pkg/pulsar/update_options.go rename to pulsaradmin/pkg/pulsar/utils/update_options.go index dc93279fee..d78fccfcec 100644 --- a/pulsaradmin/pkg/pulsar/update_options.go +++ b/pulsaradmin/pkg/pulsar/utils/update_options.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils // Options while updating the sink type UpdateOptions struct { diff --git a/pulsaradmin/pkg/pulsar/utils.go b/pulsaradmin/pkg/pulsar/utils/utils.go similarity index 91% rename from pulsaradmin/pkg/pulsar/utils.go rename to pulsaradmin/pkg/pulsar/utils/utils.go index 249f18d23f..e371b6a151 100644 --- a/pulsaradmin/pkg/pulsar/utils.go +++ b/pulsaradmin/pkg/pulsar/utils/utils.go @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils import ( "fmt" ) -func makeHTTPPath(apiVersion string, componentPath string) string { +func MakeHTTPPath(apiVersion string, componentPath string) string { return fmt.Sprintf("/admin/%s%s", apiVersion, componentPath) } diff --git a/pulsaradmin/pkg/pulsar/window_confing.go b/pulsaradmin/pkg/pulsar/utils/window_confing.go similarity index 98% rename from pulsaradmin/pkg/pulsar/window_confing.go rename to pulsaradmin/pkg/pulsar/utils/window_confing.go index d1235f8d1f..4c1bb9262d 100644 --- a/pulsaradmin/pkg/pulsar/window_confing.go +++ b/pulsaradmin/pkg/pulsar/utils/window_confing.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils const WindowConfigKey = "__WINDOWCONFIGS__" diff --git a/pulsaradmin/pkg/pulsar/worker_info.go b/pulsaradmin/pkg/pulsar/utils/worker_info.go similarity index 98% rename from pulsaradmin/pkg/pulsar/worker_info.go rename to pulsaradmin/pkg/pulsar/utils/worker_info.go index f98cc797ec..bd0dd806c0 100644 --- a/pulsaradmin/pkg/pulsar/worker_info.go +++ b/pulsaradmin/pkg/pulsar/utils/worker_info.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package utils type WorkerInfo struct { WorkerID string `json:"workerId"` From 3328f85c6c5226d0e1a78f174dc181bf74d5c149 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Tue, 5 Nov 2019 20:09:25 +0800 Subject: [PATCH 144/348] [Issue:128] Fix deploying python function (streamnative/pulsarctl#130) Signed-off-by: xiaolong.ran Fixes: streamnative/pulsarctl#128 Fixes: streamnative/pulsarctl#129 ### Motivation In go language, the `string` type default value is not same with java. --- pulsaradmin/pkg/pulsar/utils/function_confg.go | 16 ++++++++-------- pulsaradmin/pkg/pulsar/utils/window_confing.go | 18 +++++++++--------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/utils/function_confg.go b/pulsaradmin/pkg/pulsar/utils/function_confg.go index 8b312f28f9..fc663aadc1 100644 --- a/pulsaradmin/pkg/pulsar/utils/function_confg.go +++ b/pulsaradmin/pkg/pulsar/utils/function_confg.go @@ -31,7 +31,7 @@ type FunctionConfig struct { RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` AutoAck bool `json:"autoAck" yaml:"autoAck"` Parallelism int `json:"parallelism" yaml:"parallelism"` - MaxMessageRetries int `json:"maxMessageRetries" yaml:"maxMessageRetries"` + MaxMessageRetries *int `json:"maxMessageRetries" yaml:"maxMessageRetries"` Output string `json:"output" yaml:"output"` @@ -42,13 +42,13 @@ type FunctionConfig struct { // Represents either a builtin schema type (eg: 'avro', 'json', etc) or the class name for a Schema implementation OutputSchemaType string `json:"outputSchemaType" yaml:"outputSchemaType"` - Runtime string `json:"runtime" yaml:"runtime"` - DeadLetterTopic string `json:"deadLetterTopic" yaml:"deadLetterTopic"` - SubName string `json:"subName" yaml:"subName"` - FQFN string `json:"fqfn" yaml:"fqfn"` - Jar string `json:"jar" yaml:"jar"` - Py string `json:"py" yaml:"py"` - Go string `json:"go" yaml:"go"` + Runtime string `json:"runtime" yaml:"runtime"` + DeadLetterTopic string `json:"deadLetterTopic" yaml:"deadLetterTopic"` + SubName string `json:"subName" yaml:"subName"` + FQFN string `json:"fqfn" yaml:"fqfn"` + Jar *string `json:"jar" yaml:"jar"` + Py *string `json:"py" yaml:"py"` + Go *string `json:"go" yaml:"go"` // Any flags that you want to pass to the runtime. // note that in thread mode, these flags will have no impact RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` diff --git a/pulsaradmin/pkg/pulsar/utils/window_confing.go b/pulsaradmin/pkg/pulsar/utils/window_confing.go index 4c1bb9262d..cc1cae6a72 100644 --- a/pulsaradmin/pkg/pulsar/utils/window_confing.go +++ b/pulsaradmin/pkg/pulsar/utils/window_confing.go @@ -20,15 +20,15 @@ package utils const WindowConfigKey = "__WINDOWCONFIGS__" type WindowConfig struct { - WindowLengthCount int - WindowLengthDurationMs int64 - SlidingIntervalCount int - SlidingIntervalDurationMs int64 - LateDataTopic string - MaxLagMs int64 - WatermarkEmitIntervalMs int64 - TimestampExtractorClassName string - ActualWindowFunctionClassName string + WindowLengthCount *int `json:"windowLengthCount" yaml:"windowLengthCount"` + WindowLengthDurationMs *int64 `json:"windowLengthDurationMs" yaml:"windowLengthDurationMs"` + SlidingIntervalCount *int `json:"slidingIntervalCount" yaml:"slidingIntervalCount"` + SlidingIntervalDurationMs *int64 `json:"slidingIntervalDurationMs" yaml:"slidingIntervalDurationMs"` + LateDataTopic *string `json:"lateDataTopic" yaml:"lateDataTopic"` + MaxLagMs *int64 `json:"maxLagMs" yaml:"maxLagMs"` + WatermarkEmitIntervalMs *int64 `json:"watermarkEmitIntervalMs" yaml:"watermarkEmitIntervalMs"` + TimestampExtractorClassName *string `json:"timestampExtractorClassName" yaml:"timestampExtractorClassName"` + ActualWindowFunctionClassName *string `json:"actualWindowFunctionClassName" yaml:"actualWindowFunctionClassName"` } func NewDefaultWindowConfing() *WindowConfig { From 4ae5e142d51c894070516257386155146dd0cccc Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 11 Nov 2019 10:45:23 +0800 Subject: [PATCH 145/348] Move description.go to cmdutils (streamnative/pulsarctl#133) --- *Motivation* Move the description.go to cmdutils make other commands can use. *Modifications* - move file --- pulsaradmin/pkg/pulsar/common/descriptions.go | 88 ------------------- .../pkg/pulsar/common/descriptions_test.go | 67 -------------- 2 files changed, 155 deletions(-) delete mode 100644 pulsaradmin/pkg/pulsar/common/descriptions.go delete mode 100644 pulsaradmin/pkg/pulsar/common/descriptions_test.go diff --git a/pulsaradmin/pkg/pulsar/common/descriptions.go b/pulsaradmin/pkg/pulsar/common/descriptions.go deleted file mode 100644 index 59856db5f0..0000000000 --- a/pulsaradmin/pkg/pulsar/common/descriptions.go +++ /dev/null @@ -1,88 +0,0 @@ -// 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 common - -import ( - "strings" -) - -var SPACES = " " -var USEDFOR = "USED FOR:" -var PERMISSION = "REQUIRED PERMISSION:" -var EXAMPLES = "EXAMPLES:" -var OUTPUT = "OUTPUT:" -var SCOPE = "SCOPE:" - -type LongDescription struct { - CommandUsedFor string - CommandPermission string - CommandExamples []Example - CommandOutput []Output - CommandScope string -} - -type Example struct { - Desc string - Command string -} - -type Output struct { - Desc string - Out string -} - -func (desc *LongDescription) ToString() string { - out := USEDFOR + "\n" + - SPACES + desc.CommandUsedFor + "\n\n" + - PERMISSION + "\n" + - SPACES + desc.CommandPermission + "\n\n" + - OUTPUT + "\n" + - desc.outputToString() - - if desc.CommandScope != "" { - out += SCOPE + "\n" + - SPACES + desc.CommandScope + "\n\n" - } - - return out -} - -func (desc *LongDescription) ExampleToString() string { - var result string - for _, v := range desc.CommandExamples { - result += SPACES + "#" + v.Desc + "\n" + SPACES + v.Command + "\n\n" - } - return result -} - -func (desc *LongDescription) outputToString() string { - var result string - for _, v := range desc.CommandOutput { - result += SPACES + "#" + v.Desc + "\n" + makeSpace(v.Out) + "\n" - } - return result -} - -func makeSpace(s string) string { - var res string - lines := strings.Split(s, "\n") - for _, l := range lines { - res += SPACES + l + "\n" - } - return res -} diff --git a/pulsaradmin/pkg/pulsar/common/descriptions_test.go b/pulsaradmin/pkg/pulsar/common/descriptions_test.go deleted file mode 100644 index 661bb41f88..0000000000 --- a/pulsaradmin/pkg/pulsar/common/descriptions_test.go +++ /dev/null @@ -1,67 +0,0 @@ -// 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 common - -import ( - "testing" - - "github.com/stretchr/testify/assert" -) - -func TestLongDescription_exampleToString(t *testing.T) { - desc := LongDescription{} - example := Example{ - Desc: "command description", - Command: "command", - } - desc.CommandExamples = []Example{example} - res := desc.ExampleToString() - - expect := " #command description\n" + - " command\n\n" - - assert.Equal(t, expect, res) -} - -func TestLongDescription_ToString(t *testing.T) { - desc := LongDescription{} - desc.CommandUsedFor = "command used for" - desc.CommandPermission = "command permission" - example := Example{} - example.Desc = "command description" - example.Command = "command" - desc.CommandExamples = []Example{example} - out := Output{ - Desc: "Output", - Out: "Out line 1\nOut line 2", - } - desc.CommandOutput = []Output{out} - - expect := "USED FOR:\n" + - " " + desc.CommandUsedFor + "\n\n" + - "REQUIRED PERMISSION:\n" + - " " + desc.CommandPermission + "\n\n" + - "OUTPUT:\n" + - " " + "#" + out.Desc + "\n" + - " " + "Out line 1" + "\n" + - " " + "Out line 2" + "\n\n" - - result := desc.ToString() - - assert.Equal(t, expect, result) -} From 91cd5d4f74203f49f141763cfee1ec62d75faaa2 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 13 Nov 2019 15:42:20 +0800 Subject: [PATCH 146/348] [BK-SUPPORT-PART-1] Separate HTTP client and admin client (streamnative/pulsarctl#132) Master Issue: streamnative/pulsarctl#127 *Motivation* Pulsarctl needs to support bookie client API, and will use the same HTTP client to request a different server. So we need to separate the HTTP client to make the different admin client request to different servers. *Modifications* - separate the HTTP client from admin client --- pulsaradmin/pkg/auth/auth_provider.go | 27 ++ pulsaradmin/pkg/auth/tls.go | 67 ++- pulsaradmin/pkg/auth/token.go | 26 +- pulsaradmin/pkg/cli/client.go | 381 +++++++++++++++ pulsaradmin/pkg/cli/errors.go | 37 ++ pulsaradmin/pkg/pulsar/admin.go | 459 ++---------------- pulsaradmin/pkg/pulsar/admin_config.go | 56 +++ pulsaradmin/pkg/pulsar/broker_stats.go | 26 +- pulsaradmin/pkg/pulsar/brokers.go | 42 +- pulsaradmin/pkg/pulsar/cluster.go | 52 +- pulsaradmin/pkg/pulsar/functions.go | 86 ++-- pulsaradmin/pkg/pulsar/functions_worker.go | 26 +- pulsaradmin/pkg/pulsar/namespace.go | 278 +++++------ pulsaradmin/pkg/pulsar/ns_isolation_policy.go | 30 +- pulsaradmin/pkg/pulsar/resource_quotas.go | 26 +- pulsaradmin/pkg/pulsar/schema.go | 26 +- pulsaradmin/pkg/pulsar/sinks.go | 74 +-- pulsaradmin/pkg/pulsar/sources.go | 74 +-- pulsaradmin/pkg/pulsar/subscription.go | 58 +-- pulsaradmin/pkg/pulsar/tenant.go | 26 +- pulsaradmin/pkg/pulsar/topic.go | 96 ++-- 21 files changed, 1063 insertions(+), 910 deletions(-) create mode 100644 pulsaradmin/pkg/auth/auth_provider.go create mode 100644 pulsaradmin/pkg/cli/client.go create mode 100644 pulsaradmin/pkg/cli/errors.go create mode 100644 pulsaradmin/pkg/pulsar/admin_config.go diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go new file mode 100644 index 0000000000..f4d595ea8c --- /dev/null +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -0,0 +1,27 @@ +// 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 auth + +// Provider provide a general method to add auth message +type Provider interface { + // HasDataForHTTP is used to check if data for HTTP are available + HasDataForHTTP() bool + + // GetHTTPHeaders is used to get all auth headers + GetHTTPHeaders() (map[string]string, error) +} diff --git a/pulsaradmin/pkg/auth/tls.go b/pulsaradmin/pkg/auth/tls.go index ebe33d6839..1f2a406f26 100644 --- a/pulsaradmin/pkg/auth/tls.go +++ b/pulsaradmin/pkg/auth/tls.go @@ -17,26 +17,28 @@ package auth -import "crypto/tls" +import ( + "crypto/tls" + "crypto/x509" + "io/ioutil" -type TLSAuthProvider struct { - certificatePath string - privateKeyPath string -} + "github.com/pkg/errors" +) -// NewAuthenticationTLSWithParams initialize the authentication provider with map param. -func NewAuthenticationTLSWithParams(params map[string]string) *TLSAuthProvider { - return NewAuthenticationTLS( - params["tlsCertFile"], - params["tlsKeyFile"], - ) +type TLSAuthProvider struct { + certificatePath string + privateKeyPath string + allowInsecureConnection bool } // NewAuthenticationTLS initialize the authentication provider -func NewAuthenticationTLS(certificatePath string, privateKeyPath string) *TLSAuthProvider { +func NewAuthenticationTLS(certificatePath string, privateKeyPath string, + allowInsecureConnection bool) *TLSAuthProvider { + return &TLSAuthProvider{ - certificatePath: certificatePath, - privateKeyPath: privateKeyPath, + certificatePath: certificatePath, + privateKeyPath: privateKeyPath, + allowInsecureConnection: allowInsecureConnection, } } @@ -54,3 +56,40 @@ func (p *TLSAuthProvider) GetTLSCertificate() (*tls.Certificate, error) { cert, err := tls.LoadX509KeyPair(p.certificatePath, p.privateKeyPath) return &cert, err } + +func (p *TLSAuthProvider) GetTLSConfig(certFile string, allowInsecureConnection bool) (*tls.Config, error) { + tlsConfig := &tls.Config{ + InsecureSkipVerify: allowInsecureConnection, + } + + if certFile != "" { + caCerts, err := ioutil.ReadFile(certFile) + if err != nil { + return nil, err + } + + tlsConfig.RootCAs = x509.NewCertPool() + if !tlsConfig.RootCAs.AppendCertsFromPEM(caCerts) { + return nil, errors.New("failed to parse root CAs certificates") + } + } + + cert, err := p.GetTLSCertificate() + if err != nil { + return nil, err + } + + if cert != nil { + tlsConfig.Certificates = []tls.Certificate{*cert} + } + + return tlsConfig, nil +} + +func (p *TLSAuthProvider) HasDataForHTTP() bool { + return false +} + +func (p *TLSAuthProvider) GetHTTPHeaders() (map[string]string, error) { + return nil, errors.New("Unsupported operation") +} diff --git a/pulsaradmin/pkg/auth/token.go b/pulsaradmin/pkg/auth/token.go index c928ed585c..008b1825b7 100644 --- a/pulsaradmin/pkg/auth/token.go +++ b/pulsaradmin/pkg/auth/token.go @@ -28,18 +28,6 @@ type TokenAuthProvider struct { tokenSupplier func() (string, error) } -// NewAuthenticationTokenWithParams return a interface of Provider with string map. -func NewAuthenticationTokenWithParams(params map[string]string) (*TokenAuthProvider, error) { - switch { - case params["token"] != "": - return NewAuthenticationToken(params["token"]), nil - case params["file"] != "": - return NewAuthenticationTokenFromFile(params["file"]), nil - default: - return nil, errors.New("missing configuration for token auth") - } -} - // NewAuthenticationToken return a interface of Provider with a string token. func NewAuthenticationToken(token string) *TokenAuthProvider { return &TokenAuthProvider{ @@ -83,3 +71,17 @@ func (p *TokenAuthProvider) GetData() ([]byte, error) { } return []byte(t), nil } + +func (p *TokenAuthProvider) HasDataForHTTP() bool { + return true +} + +func (p *TokenAuthProvider) GetHTTPHeaders() (map[string]string, error) { + data, err := p.GetData() + if err != nil { + return nil, err + } + headers := make(map[string]string) + headers["Authorization"] = "Bearer " + string(data) + return headers, nil +} diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go new file mode 100644 index 0000000000..e55988dfa9 --- /dev/null +++ b/pulsaradmin/pkg/cli/client.go @@ -0,0 +1,381 @@ +// 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 cli + +import ( + "bytes" + "encoding/json" + "io" + "io/ioutil" + "net/http" + "net/url" + "path" + + "github.com/streamnative/pulsar-admin-go/pkg/auth" +) + +// Client is a base client that is used to make http request to the ServiceURL +type Client struct { + ServiceURL string + HTTPClient *http.Client + VersionInfo string + AuthProvider auth.Provider +} + +func (c *Client) newRequest(method, path string) (*request, error) { + base, _ := url.Parse(c.ServiceURL) + u, err := url.Parse(path) + if err != nil { + return nil, err + } + + req := &request{ + method: method, + url: &url.URL{ + Scheme: base.Scheme, + User: base.User, + Host: base.Host, + Path: endpoint(base.Path, u.Path), + }, + params: make(url.Values), + } + return req, nil +} + +func (c *Client) doRequest(r *request) (*http.Response, error) { + req, err := r.toHTTP() + if err != nil { + return nil, err + } + + if r.contentType != "" { + req.Header.Set("Content-Type", r.contentType) + } else { + // add default headers + req.Header.Set("Content-Type", "application/json") + } + req.Header.Set("Accept", "application/json") + req.Header.Set("User-Agent", c.useragent()) + + if c.AuthProvider != nil { + if c.AuthProvider.HasDataForHTTP() { + headers, err := c.AuthProvider.GetHTTPHeaders() + if err != nil { + return nil, err + } + for k, v := range headers { + req.Header.Set(k, v) + } + } + } + + hc := c.HTTPClient + if hc == nil { + hc = http.DefaultClient + } + + return hc.Do(req) +} + +// MakeRequest can make a simple request and handle the response by yourself +func (c *Client) MakeRequest(method, endpoint string) (*http.Response, error) { + req, err := c.newRequest(method, endpoint) + if err != nil { + return nil, err + } + + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return nil, err + } + + return resp, nil +} + +func (c *Client) Get(endpoint string, obj interface{}) error { + _, err := c.GetWithQueryParams(endpoint, obj, nil, true) + return err +} + +func (c *Client) GetWithQueryParams(endpoint string, obj interface{}, params map[string]string, + decode bool) ([]byte, error) { + + req, err := c.newRequest(http.MethodGet, endpoint) + if err != nil { + return nil, err + } + + if params != nil { + query := req.url.Query() + for k, v := range params { + query.Add(k, v) + } + req.params = query + } + + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return nil, err + } + defer safeRespClose(resp) + + if obj != nil { + if err := decodeJSONBody(resp, &obj); err != nil { + return nil, err + } + } else if !decode { + body, err := ioutil.ReadAll(resp.Body) + if err != nil { + return nil, err + } + return body, err + } + + return nil, err +} + +func (c *Client) useragent() string { + return c.VersionInfo +} + +func (c *Client) Put(endpoint string, in interface{}) error { + return c.PutWithQueryParams(endpoint, in, nil, nil) +} + +func (c *Client) PutWithQueryParams(endpoint string, in, obj interface{}, params map[string]string) error { + req, err := c.newRequest(http.MethodPut, endpoint) + if err != nil { + return err + } + req.obj = in + + if params != nil { + query := req.url.Query() + for k, v := range params { + query.Add(k, v) + } + req.params = query + } + + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + + if obj != nil { + if err := decodeJSONBody(resp, &obj); err != nil { + return err + } + } + + return nil +} + +func (c *Client) PutWithMultiPart(endpoint string, body io.Reader, contentType string) error { + req, err := c.newRequest(http.MethodPut, endpoint) + if err != nil { + return err + } + req.body = body + req.contentType = contentType + + // nolint + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + + return nil +} + +func (c *Client) Delete(endpoint string) error { + return c.DeleteWithQueryParams(endpoint, nil) +} + +func (c *Client) DeleteWithQueryParams(endpoint string, params map[string]string) error { + req, err := c.newRequest(http.MethodDelete, endpoint) + if err != nil { + return err + } + + if params != nil { + query := req.url.Query() + for k, v := range params { + query.Add(k, v) + } + req.params = query + } + + // nolint + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + + return nil +} + +func (c *Client) Post(endpoint string, in interface{}) error { + return c.PostWithObj(endpoint, in, nil) +} + +func (c *Client) PostWithObj(endpoint string, in, obj interface{}) error { + req, err := c.newRequest(http.MethodPost, endpoint) + if err != nil { + return err + } + req.obj = in + + // nolint + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + if obj != nil { + if err := decodeJSONBody(resp, &obj); err != nil { + return err + } + } + + return nil +} + +func (c *Client) PostWithMultiPart(endpoint string, in interface{}, body io.Reader, contentType string) error { + req, err := c.newRequest(http.MethodPost, endpoint) + if err != nil { + return err + } + req.obj = in + req.body = body + req.contentType = contentType + + // nolint + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + + return nil +} + +type request struct { + method string + contentType string + url *url.URL + params url.Values + + obj interface{} + body io.Reader +} + +func (r *request) toHTTP() (*http.Request, error) { + r.url.RawQuery = r.params.Encode() + + // add a request body if there is one + if r.body == nil && r.obj != nil { + body, err := encodeJSONBody(r.obj) + if err != nil { + return nil, err + } + r.body = body + } + + req, err := http.NewRequest(r.method, r.url.RequestURI(), r.body) + if err != nil { + return nil, err + } + + req.URL.Host = r.url.Host + req.URL.Scheme = r.url.Scheme + req.Host = r.url.Host + return req, nil +} + +// respIsOk is used to validate a successful http status code +func respIsOk(resp *http.Response) bool { + return resp.StatusCode >= http.StatusOK && resp.StatusCode <= http.StatusNoContent +} + +// checkSuccessful checks for a valid response and parses an error +func checkSuccessful(resp *http.Response, err error) (*http.Response, error) { + if err != nil { + safeRespClose(resp) + return nil, err + } + + if !respIsOk(resp) { + defer safeRespClose(resp) + return nil, responseError(resp) + } + + return resp, nil +} + +func endpoint(parts ...string) string { + return path.Join(parts...) +} + +// encodeJSONBody is used to JSON encode a body +func encodeJSONBody(obj interface{}) (io.Reader, error) { + buf := bytes.NewBuffer(nil) + enc := json.NewEncoder(buf) + if err := enc.Encode(obj); err != nil { + return nil, err + } + return buf, nil +} + +// decodeJSONBody is used to JSON decode a body +func decodeJSONBody(resp *http.Response, out interface{}) error { + dec := json.NewDecoder(resp.Body) + return dec.Decode(out) +} + +// safeRespClose is used to close a response body +func safeRespClose(resp *http.Response) { + if resp != nil { + // ignore error since it is closing a response body + _ = resp.Body.Close() + } +} + +// responseError is used to parse a response into a pulsar error +func responseError(resp *http.Response) error { + var e Error + body, err := ioutil.ReadAll(resp.Body) + if err != nil { + e.Reason = err.Error() + e.Code = resp.StatusCode + return e + } + + json.Unmarshal(body, &e) + + e.Code = resp.StatusCode + + if e.Reason == "" { + e.Reason = unknownErrorReason + } + + return e +} diff --git a/pulsaradmin/pkg/cli/errors.go b/pulsaradmin/pkg/cli/errors.go new file mode 100644 index 0000000000..3a7e91a4c9 --- /dev/null +++ b/pulsaradmin/pkg/cli/errors.go @@ -0,0 +1,37 @@ +// 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 cli + +import "fmt" + +const unknownErrorReason = "Unknown pulsar error" + +// Error is a admin error type +type Error struct { + Reason string `json:"reason"` + Code int +} + +func (e Error) Error() string { + return fmt.Sprintf("code: %d reason: %s", e.Code, e.Reason) +} + +func IsAdminError(err error) bool { + _, ok := err.(Error) + return ok +} diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index bf481627f0..8b627caf8a 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -18,64 +18,21 @@ package pulsar import ( - "bytes" - "crypto/tls" - "crypto/x509" - "encoding/json" - "errors" - "io" - "io/ioutil" "net/http" - "net/url" "path" "strings" - "time" "github.com/streamnative/pulsar-admin-go/pkg/auth" + "github.com/streamnative/pulsar-admin-go/pkg/cli" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) -const ( - DefaultWebServiceURL = "http://localhost:8080" - DefaultHTTPTimeOutDuration = 5 * time.Minute -) - -var ReleaseVersion = "None" - -// Config is used to configure the admin client -type Config struct { - WebServiceURL string - HTTPTimeout time.Duration - HTTPClient *http.Client - APIVersion common.APIVersion - - Auth *auth.TLSAuthProvider - AuthParams string - TLSOptions *TLSOptions - TokenAuth *auth.TokenAuthProvider -} - type TLSOptions struct { TrustCertsFilePath string AllowInsecureConnection bool } -// DefaultConfig returns a default configuration for the pulsar admin client -func DefaultConfig() *Config { - config := &Config{ - WebServiceURL: DefaultWebServiceURL, - HTTPClient: &http.Client{ - Timeout: DefaultHTTPTimeOutDuration, - }, - - TLSOptions: &TLSOptions{ - AllowInsecureConnection: false, - }, - } - return config -} - // Client provides a client to the Pulsar Restful API type Client interface { Clusters() Clusters @@ -94,19 +51,9 @@ type Client interface { FunctionsWorker() FunctionsWorker } -type client struct { - webServiceURL string - apiVersion string - httpClient *http.Client - versionInfo string - - // TLS config - auth *auth.TLSAuthProvider - authParams string - tlsOptions *TLSOptions - transport *http.Transport - - tokenAuth *auth.TokenAuthProvider +type pulsarClient struct { + Client *cli.Client + APIVersion common.APIVersion } // New returns a new client @@ -115,405 +62,65 @@ func New(config *Config) (Client, error) { config.WebServiceURL = DefaultWebServiceURL } - c := &client{ - apiVersion: config.APIVersion.String(), - webServiceURL: config.WebServiceURL, - versionInfo: ReleaseVersion, - tokenAuth: config.TokenAuth, - } - - if strings.HasPrefix(c.webServiceURL, "https://") { - c.authParams = config.AuthParams - c.tlsOptions = config.TLSOptions - mapAuthParams := make(map[string]string) - - err := json.Unmarshal([]byte(c.authParams), &mapAuthParams) - if err != nil { - return nil, err - } - c.auth = auth.NewAuthenticationTLSWithParams(mapAuthParams) - - tlsConf, err := c.getTLSConfig() - if err != nil { - return nil, err - } - - c.transport = &http.Transport{ - TLSHandshakeTimeout: 15 * time.Second, - MaxIdleConnsPerHost: 10, - TLSClientConfig: tlsConf, - } - } - - return c, nil -} - -func (c *client) getTLSConfig() (*tls.Config, error) { - tlsConfig := &tls.Config{ - InsecureSkipVerify: c.tlsOptions.AllowInsecureConnection, - } - - if c.tlsOptions.TrustCertsFilePath != "" { - caCerts, err := ioutil.ReadFile(c.tlsOptions.TrustCertsFilePath) - if err != nil { - return nil, err - } - - tlsConfig.RootCAs = x509.NewCertPool() - ok := tlsConfig.RootCAs.AppendCertsFromPEM(caCerts) - if !ok { - return nil, errors.New("failed to parse root CAs certificates") - } + c := &pulsarClient{ + APIVersion: config.APIVersion, + Client: &cli.Client{ + ServiceURL: config.WebServiceURL, + VersionInfo: ReleaseVersion, + HTTPClient: &http.Client{ + Timeout: DefaultHTTPTimeOutDuration, + }, + }, } - cert, err := c.auth.GetTLSCertificate() + err := c.initAuth(config) if err != nil { return nil, err } - if cert != nil { - tlsConfig.Certificates = []tls.Certificate{*cert} - } - - return tlsConfig, nil -} - -func (c *client) endpoint(componentPath string, parts ...string) string { - return path.Join(utils.MakeHTTPPath(c.apiVersion, componentPath), endpoint(parts...)) + return c, nil } -// get is used to do a GET request against an endpoint -// and deserialize the response into an interface - -func (c *client) getWithQueryParams(endpoint string, obj interface{}, params map[string]string, - decode bool) ([]byte, error) { - - req, err := c.newRequest(http.MethodGet, endpoint) - if err != nil { - return nil, err - } - - if params != nil { - query := req.url.Query() - for k, v := range params { - query.Add(k, v) - } - req.params = query - } - - resp, err := checkSuccessful(c.doRequest(req)) - if err != nil { - return nil, err - } - defer safeRespClose(resp) - - if obj != nil { - if err := decodeJSONBody(resp, &obj); err != nil { - return nil, err - } - } else if !decode { - body, err := ioutil.ReadAll(resp.Body) +func (c *pulsarClient) initAuth(config *Config) error { + if strings.HasPrefix(config.WebServiceURL, "https") { + err := c.initTLS(config) if err != nil { - return nil, err - } - return body, err - } - - return nil, err -} - -func (c *client) get(endpoint string, obj interface{}) error { - _, err := c.getWithQueryParams(endpoint, obj, nil, true) - return err -} - -func (c *client) put(endpoint string, in interface{}) error { - return c.putWithQueryParams(endpoint, in, nil, nil) -} - -func (c *client) putWithQueryParams(endpoint string, in, obj interface{}, params map[string]string) error { - req, err := c.newRequest(http.MethodPut, endpoint) - if err != nil { - return err - } - req.obj = in - - if params != nil { - query := req.url.Query() - for k, v := range params { - query.Add(k, v) - } - req.params = query - } - - resp, err := checkSuccessful(c.doRequest(req)) - if err != nil { - return err - } - defer safeRespClose(resp) - - if obj != nil { - if err := decodeJSONBody(resp, &obj); err != nil { return err } } - return nil -} - -func (c *client) delete(endpoint string) error { - return c.deleteWithQueryParams(endpoint, nil, nil) -} - -func (c *client) deleteWithQueryParams(endpoint string, obj interface{}, params map[string]string) error { - req, err := c.newRequest(http.MethodDelete, endpoint) - if err != nil { - return err - } - - if params != nil { - query := req.url.Query() - for k, v := range params { - query.Add(k, v) - } - req.params = query - } - - resp, err := checkSuccessful(c.doRequest(req)) - if err != nil { - return err - } - defer safeRespClose(resp) - - if obj != nil { - if err := decodeJSONBody(resp, &obj); err != nil { - return err - } + if config.TokenFile != "" || config.Token != "" { + c.initToken(config) } return nil } -func (c *client) post(endpoint string, in interface{}) error { - return c.postWithObj(endpoint, in, nil) -} - -func (c *client) postWithObj(endpoint string, in, obj interface{}) error { - req, err := c.newRequest(http.MethodPost, endpoint) - if err != nil { - return err - } - req.obj = in - - // nolint - resp, err := checkSuccessful(c.doRequest(req)) - if err != nil { - return err - } - defer safeRespClose(resp) - if obj != nil { - if err := decodeJSONBody(resp, &obj); err != nil { - return err - } - } - - return nil -} - -func (c *client) putWithMultiPart(endpoint string, body io.Reader, contentType string) error { - req, err := c.newRequest(http.MethodPut, endpoint) - if err != nil { - return err - } - req.body = body - req.contentType = contentType - - // nolint - resp, err := checkSuccessful(c.doRequest(req)) - if err != nil { - return err - } - defer safeRespClose(resp) - - return nil -} - -func (c *client) postWithMultiPart(endpoint string, in interface{}, body io.Reader, contentType string) error { - req, err := c.newRequest(http.MethodPost, endpoint) +func (c *pulsarClient) initTLS(config *Config) error { + tlsAuth := auth.NewAuthenticationTLS(config.TLSCertFile, config.TLSKeyFile, config.TLSAllowInsecureConnection) + tlsConf, err := tlsAuth.GetTLSConfig(config.TLSCertFile, config.TLSAllowInsecureConnection) if err != nil { return err } - req.obj = in - req.body = body - req.contentType = contentType - // nolint - resp, err := checkSuccessful(c.doRequest(req)) - if err != nil { - return err + c.Client.HTTPClient.Transport = &http.Transport{ + MaxIdleConnsPerHost: 10, + TLSClientConfig: tlsConf, } - defer safeRespClose(resp) return nil } -type request struct { - method string - contentType string - url *url.URL - params url.Values - - obj interface{} - body io.Reader -} - -func (r *request) toHTTP() (*http.Request, error) { - r.url.RawQuery = r.params.Encode() - - // add a request body if there is one - if r.body == nil && r.obj != nil { - body, err := encodeJSONBody(r.obj) - if err != nil { - return nil, err - } - r.body = body - } - - req, err := http.NewRequest(r.method, r.url.RequestURI(), r.body) - if err != nil { - return nil, err - } - - req.URL.Host = r.url.Host - req.URL.Scheme = r.url.Scheme - req.Host = r.url.Host - return req, nil -} - -func (c *client) newRequest(method, path string) (*request, error) { - base, _ := url.Parse(c.webServiceURL) - u, err := url.Parse(path) - if err != nil { - return nil, err - } - - req := &request{ - method: method, - url: &url.URL{ - Scheme: base.Scheme, - User: base.User, - Host: base.Host, - Path: endpoint(base.Path, u.Path), - }, - params: make(url.Values), - } - return req, nil -} - -func (c *client) useragent() string { - return c.versionInfo -} - -func (c *client) doRequest(r *request) (*http.Response, error) { - req, err := r.toHTTP() - if err != nil { - return nil, err - } - - if r.contentType != "" { - req.Header.Set("Content-Type", r.contentType) - } else { - // add default headers - req.Header.Set("Content-Type", "application/json") - req.Header.Set("Accept", "application/json") - } - - if c.tokenAuth != nil { - data, _ := c.tokenAuth.GetData() - req.Header.Set("Authorization", "Bearer "+string(data)) - } - - req.Header.Set("User-Agent", c.useragent()) - - hc := c.httpClient - if hc == nil { - hc = &http.Client{ - Timeout: DefaultHTTPTimeOutDuration, - } - } - - if c.transport != nil { - hc.Transport = c.transport - } - - return hc.Do(req) -} - -// encodeJSONBody is used to JSON encode a body -func encodeJSONBody(obj interface{}) (io.Reader, error) { - buf := bytes.NewBuffer(nil) - enc := json.NewEncoder(buf) - if err := enc.Encode(obj); err != nil { - return nil, err - } - return buf, nil -} - -// decodeJSONBody is used to JSON decode a body -func decodeJSONBody(resp *http.Response, out interface{}) error { - dec := json.NewDecoder(resp.Body) - return dec.Decode(out) -} - -// safeRespClose is used to close a response body -func safeRespClose(resp *http.Response) { - if resp != nil { - // ignore error since it is closing a response body - _ = resp.Body.Close() +func (c *pulsarClient) initToken(config *Config) { + if config.Token != "" { + c.Client.AuthProvider = auth.NewAuthenticationToken(config.Token) } -} - -// responseError is used to parse a response into a pulsar error -func responseError(resp *http.Response) error { - var e common.Error - body, err := ioutil.ReadAll(resp.Body) - if err != nil { - e.Reason = err.Error() - e.Code = resp.StatusCode - return e - } - - json.Unmarshal(body, &e) - - e.Code = resp.StatusCode - if e.Reason == "" { - e.Reason = common.UnknownErrorReason + if config.TokenFile != "" { + c.Client.AuthProvider = auth.NewAuthenticationTokenFromFile(config.TokenFile) } - - return e -} - -// respIsOk is used to validate a successful http status code -func respIsOk(resp *http.Response) bool { - return resp.StatusCode >= http.StatusOK && resp.StatusCode <= http.StatusNoContent -} - -// checkSuccessful checks for a valid response and parses an error -func checkSuccessful(resp *http.Response, err error) (*http.Response, error) { - if err != nil { - safeRespClose(resp) - return nil, err - } - - if !respIsOk(resp) { - defer safeRespClose(resp) - return nil, responseError(resp) - } - - return resp, nil } -func endpoint(parts ...string) string { - return path.Join(parts...) +func (c *pulsarClient) endpoint(componentPath string, parts ...string) string { + return path.Join(utils.MakeHTTPPath(c.APIVersion.String(), componentPath), path.Join(parts...)) } diff --git a/pulsaradmin/pkg/pulsar/admin_config.go b/pulsaradmin/pkg/pulsar/admin_config.go new file mode 100644 index 0000000000..ff4bdcf771 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/admin_config.go @@ -0,0 +1,56 @@ +// 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 pulsar + +import ( + "time" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" +) + +const ( + DefaultWebServiceURL = "http://localhost:8080" + DefaultHTTPTimeOutDuration = 5 * time.Minute +) + +var ReleaseVersion = "None" + +// Config is used to configure the admin client +type Config struct { + WebServiceURL string + HTTPTimeout time.Duration + // TODO: api version should apply to the method + APIVersion common.APIVersion + + //Auth *auth.TLSAuthProvider + TLSCertFile string + TLSKeyFile string + TLSAllowInsecureConnection bool + + // Token and TokenFile is used to config the pulsarctl using token to authentication + Token string + TokenFile string +} + +// DefaultConfig returns a default configuration for the pulsar admin client +func DefaultConfig() *Config { + config := &Config{ + WebServiceURL: DefaultWebServiceURL, + } + return config +} diff --git a/pulsaradmin/pkg/pulsar/broker_stats.go b/pulsaradmin/pkg/pulsar/broker_stats.go index bcf8a17ddf..387cae9d32 100644 --- a/pulsaradmin/pkg/pulsar/broker_stats.go +++ b/pulsaradmin/pkg/pulsar/broker_stats.go @@ -40,22 +40,22 @@ type BrokerStats interface { } type brokerStats struct { - client *client + pulsar *pulsarClient basePath string } // BrokerStats is used to access the broker stats endpoints -func (c *client) BrokerStats() BrokerStats { +func (c *pulsarClient) BrokerStats() BrokerStats { return &brokerStats{ - client: c, + pulsar: c, basePath: "/broker-stats", } } func (bs *brokerStats) GetMetrics() ([]utils.Metrics, error) { - endpoint := bs.client.endpoint(bs.basePath, "/metrics") + endpoint := bs.pulsar.endpoint(bs.basePath, "/metrics") var response []utils.Metrics - err := bs.client.get(endpoint, &response) + err := bs.pulsar.Client.Get(endpoint, &response) if err != nil { return nil, err } @@ -64,9 +64,9 @@ func (bs *brokerStats) GetMetrics() ([]utils.Metrics, error) { } func (bs *brokerStats) GetMBeans() ([]utils.Metrics, error) { - endpoint := bs.client.endpoint(bs.basePath, "/mbeans") + endpoint := bs.pulsar.endpoint(bs.basePath, "/mbeans") var response []utils.Metrics - err := bs.client.get(endpoint, &response) + err := bs.pulsar.Client.Get(endpoint, &response) if err != nil { return nil, err } @@ -75,8 +75,8 @@ func (bs *brokerStats) GetMBeans() ([]utils.Metrics, error) { } func (bs *brokerStats) GetTopics() (string, error) { - endpoint := bs.client.endpoint(bs.basePath, "/topics") - buf, err := bs.client.getWithQueryParams(endpoint, nil, nil, false) + endpoint := bs.pulsar.endpoint(bs.basePath, "/topics") + buf, err := bs.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) if err != nil { return "", err } @@ -85,9 +85,9 @@ func (bs *brokerStats) GetTopics() (string, error) { } func (bs *brokerStats) GetLoadReport() (*utils.LocalBrokerData, error) { - endpoint := bs.client.endpoint(bs.basePath, "/load-report") + endpoint := bs.pulsar.endpoint(bs.basePath, "/load-report") response := utils.NewLocalBrokerData() - err := bs.client.get(endpoint, &response) + err := bs.pulsar.Client.Get(endpoint, &response) if err != nil { return nil, nil } @@ -95,9 +95,9 @@ func (bs *brokerStats) GetLoadReport() (*utils.LocalBrokerData, error) { } func (bs *brokerStats) GetAllocatorStats(allocatorName string) (*utils.AllocatorStats, error) { - endpoint := bs.client.endpoint(bs.basePath, "/allocator-stats", allocatorName) + endpoint := bs.pulsar.endpoint(bs.basePath, "/allocator-stats", allocatorName) var allocatorStats utils.AllocatorStats - err := bs.client.get(endpoint, &allocatorStats) + err := bs.pulsar.Client.Get(endpoint, &allocatorStats) if err != nil { return nil, err } diff --git a/pulsaradmin/pkg/pulsar/brokers.go b/pulsaradmin/pkg/pulsar/brokers.go index a292397a98..da8d28c3fa 100644 --- a/pulsaradmin/pkg/pulsar/brokers.go +++ b/pulsaradmin/pkg/pulsar/brokers.go @@ -58,22 +58,22 @@ type Brokers interface { } type broker struct { - client *client + pulsar *pulsarClient basePath string } // Brokers is used to access the brokers endpoints -func (c *client) Brokers() Brokers { +func (c *pulsarClient) Brokers() Brokers { return &broker{ - client: c, + pulsar: c, basePath: "/brokers", } } func (b *broker) GetActiveBrokers(cluster string) ([]string, error) { - endpoint := b.client.endpoint(b.basePath, cluster) + endpoint := b.pulsar.endpoint(b.basePath, cluster) var res []string - err := b.client.get(endpoint, &res) + err := b.pulsar.Client.Get(endpoint, &res) if err != nil { return nil, err } @@ -81,9 +81,9 @@ func (b *broker) GetActiveBrokers(cluster string) ([]string, error) { } func (b *broker) GetDynamicConfigurationNames() ([]string, error) { - endpoint := b.client.endpoint(b.basePath, "/configuration/") + endpoint := b.pulsar.endpoint(b.basePath, "/configuration/") var res []string - err := b.client.get(endpoint, &res) + err := b.pulsar.Client.Get(endpoint, &res) if err != nil { return nil, err } @@ -91,9 +91,9 @@ func (b *broker) GetDynamicConfigurationNames() ([]string, error) { } func (b *broker) GetOwnedNamespaces(cluster, brokerURL string) (map[string]utils.NamespaceOwnershipStatus, error) { - endpoint := b.client.endpoint(b.basePath, cluster, brokerURL, "ownedNamespaces") + endpoint := b.pulsar.endpoint(b.basePath, cluster, brokerURL, "ownedNamespaces") var res map[string]utils.NamespaceOwnershipStatus - err := b.client.get(endpoint, &res) + err := b.pulsar.Client.Get(endpoint, &res) if err != nil { return nil, err } @@ -102,19 +102,19 @@ func (b *broker) GetOwnedNamespaces(cluster, brokerURL string) (map[string]utils func (b *broker) UpdateDynamicConfiguration(configName, configValue string) error { value := url.QueryEscape(configValue) - endpoint := b.client.endpoint(b.basePath, "/configuration/", configName, value) - return b.client.post(endpoint, nil) + endpoint := b.pulsar.endpoint(b.basePath, "/configuration/", configName, value) + return b.pulsar.Client.Post(endpoint, nil) } func (b *broker) DeleteDynamicConfiguration(configName string) error { - endpoint := b.client.endpoint(b.basePath, "/configuration/", configName) - return b.client.delete(endpoint) + endpoint := b.pulsar.endpoint(b.basePath, "/configuration/", configName) + return b.pulsar.Client.Delete(endpoint) } func (b *broker) GetRuntimeConfigurations() (map[string]string, error) { - endpoint := b.client.endpoint(b.basePath, "/configuration/", "runtime") + endpoint := b.pulsar.endpoint(b.basePath, "/configuration/", "runtime") var res map[string]string - err := b.client.get(endpoint, &res) + err := b.pulsar.Client.Get(endpoint, &res) if err != nil { return nil, err } @@ -122,9 +122,9 @@ func (b *broker) GetRuntimeConfigurations() (map[string]string, error) { } func (b *broker) GetInternalConfigurationData() (*utils.InternalConfigurationData, error) { - endpoint := b.client.endpoint(b.basePath, "/internal-configuration") + endpoint := b.pulsar.endpoint(b.basePath, "/internal-configuration") var res utils.InternalConfigurationData - err := b.client.get(endpoint, &res) + err := b.pulsar.Client.Get(endpoint, &res) if err != nil { return nil, err } @@ -132,9 +132,9 @@ func (b *broker) GetInternalConfigurationData() (*utils.InternalConfigurationDat } func (b *broker) GetAllDynamicConfigurations() (map[string]string, error) { - endpoint := b.client.endpoint(b.basePath, "/configuration/", "values") + endpoint := b.pulsar.endpoint(b.basePath, "/configuration/", "values") var res map[string]string - err := b.client.get(endpoint, &res) + err := b.pulsar.Client.Get(endpoint, &res) if err != nil { return nil, err } @@ -142,9 +142,9 @@ func (b *broker) GetAllDynamicConfigurations() (map[string]string, error) { } func (b *broker) HealthCheck() error { - endpoint := b.client.endpoint(b.basePath, "/health") + endpoint := b.pulsar.endpoint(b.basePath, "/health") - buf, err := b.client.getWithQueryParams(endpoint, nil, nil, false) + buf, err := b.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) if err != nil { return err } diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/pulsar/cluster.go index 3f9cc4309b..a08914323f 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/pulsar/cluster.go @@ -61,82 +61,82 @@ type Clusters interface { } type clusters struct { - client *client + pulsar *pulsarClient basePath string } // Clusters is used to access the cluster endpoints. -func (c *client) Clusters() Clusters { +func (c *pulsarClient) Clusters() Clusters { return &clusters{ - client: c, + pulsar: c, basePath: "/clusters", } } func (c *clusters) List() ([]string, error) { var clusters []string - err := c.client.get(c.client.endpoint(c.basePath), &clusters) + err := c.pulsar.Client.Get(c.pulsar.endpoint(c.basePath), &clusters) return clusters, err } func (c *clusters) Get(name string) (utils.ClusterData, error) { cdata := utils.ClusterData{} - endpoint := c.client.endpoint(c.basePath, name) - err := c.client.get(endpoint, &cdata) + endpoint := c.pulsar.endpoint(c.basePath, name) + err := c.pulsar.Client.Get(endpoint, &cdata) return cdata, err } func (c *clusters) Create(cdata utils.ClusterData) error { - endpoint := c.client.endpoint(c.basePath, cdata.Name) - return c.client.put(endpoint, &cdata) + endpoint := c.pulsar.endpoint(c.basePath, cdata.Name) + return c.pulsar.Client.Put(endpoint, &cdata) } func (c *clusters) Delete(name string) error { - endpoint := c.client.endpoint(c.basePath, name) - return c.client.delete(endpoint) + endpoint := c.pulsar.endpoint(c.basePath, name) + return c.pulsar.Client.Delete(endpoint) } func (c *clusters) Update(cdata utils.ClusterData) error { - endpoint := c.client.endpoint(c.basePath, cdata.Name) - return c.client.post(endpoint, &cdata) + endpoint := c.pulsar.endpoint(c.basePath, cdata.Name) + return c.pulsar.Client.Post(endpoint, &cdata) } func (c *clusters) GetPeerClusters(name string) ([]string, error) { var peerClusters []string - endpoint := c.client.endpoint(c.basePath, name, "peers") - err := c.client.get(endpoint, &peerClusters) + endpoint := c.pulsar.endpoint(c.basePath, name, "peers") + err := c.pulsar.Client.Get(endpoint, &peerClusters) return peerClusters, err } func (c *clusters) UpdatePeerClusters(cluster string, peerClusters []string) error { - endpoint := c.client.endpoint(c.basePath, cluster, "peers") - return c.client.post(endpoint, peerClusters) + endpoint := c.pulsar.endpoint(c.basePath, cluster, "peers") + return c.pulsar.Client.Post(endpoint, peerClusters) } func (c *clusters) CreateFailureDomain(data utils.FailureDomainData) error { - endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) - return c.client.post(endpoint, &data) + endpoint := c.pulsar.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) + return c.pulsar.Client.Post(endpoint, &data) } func (c *clusters) GetFailureDomain(clusterName string, domainName string) (utils.FailureDomainData, error) { var res utils.FailureDomainData - endpoint := c.client.endpoint(c.basePath, clusterName, "failureDomains", domainName) - err := c.client.get(endpoint, &res) + endpoint := c.pulsar.endpoint(c.basePath, clusterName, "failureDomains", domainName) + err := c.pulsar.Client.Get(endpoint, &res) return res, err } func (c *clusters) ListFailureDomains(clusterName string) (utils.FailureDomainMap, error) { var domainData utils.FailureDomainMap - endpoint := c.client.endpoint(c.basePath, clusterName, "failureDomains") - err := c.client.get(endpoint, &domainData) + endpoint := c.pulsar.endpoint(c.basePath, clusterName, "failureDomains") + err := c.pulsar.Client.Get(endpoint, &domainData) return domainData, err } func (c *clusters) DeleteFailureDomain(data utils.FailureDomainData) error { - endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) - return c.client.delete(endpoint) + endpoint := c.pulsar.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) + return c.pulsar.Client.Delete(endpoint) } func (c *clusters) UpdateFailureDomain(data utils.FailureDomainData) error { - endpoint := c.client.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) - return c.client.post(endpoint, &data) + endpoint := c.pulsar.endpoint(c.basePath, data.ClusterName, "failureDomains", data.DomainName) + return c.pulsar.Client.Post(endpoint, &data) } diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index 6f5c751e36..f9ac0c9be1 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -113,14 +113,14 @@ type Functions interface { } type functions struct { - client *client + pulsar *pulsarClient basePath string } // Functions is used to access the functions endpoints -func (c *client) Functions() Functions { +func (c *pulsarClient) Functions() Functions { return &functions{ - client: c, + pulsar: c, basePath: "/functions", } } @@ -140,7 +140,7 @@ func (f *functions) createTextFromFiled(w *multipart.Writer, value string) (io.W } func (f *functions) CreateFunc(funcConf *utils.FunctionConfig, fileName string) error { - endpoint := f.client.endpoint(f.basePath, funcConf.Tenant, funcConf.Namespace, funcConf.Name) + endpoint := f.pulsar.endpoint(f.basePath, funcConf.Tenant, funcConf.Namespace, funcConf.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -190,7 +190,7 @@ func (f *functions) CreateFunc(funcConf *utils.FunctionConfig, fileName string) } contentType := multiPartWriter.FormDataContentType() - err = f.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) + err = f.pulsar.Client.PostWithMultiPart(endpoint, nil, bodyBuf, contentType) if err != nil { return err } @@ -199,7 +199,7 @@ func (f *functions) CreateFunc(funcConf *utils.FunctionConfig, fileName string) } func (f *functions) CreateFuncWithURL(funcConf *utils.FunctionConfig, pkgURL string) error { - endpoint := f.client.endpoint(f.basePath, funcConf.Tenant, funcConf.Namespace, funcConf.Name) + endpoint := f.pulsar.endpoint(f.basePath, funcConf.Tenant, funcConf.Namespace, funcConf.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -235,7 +235,7 @@ func (f *functions) CreateFuncWithURL(funcConf *utils.FunctionConfig, pkgURL str } contentType := multiPartWriter.FormDataContentType() - err = f.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) + err = f.pulsar.Client.PostWithMultiPart(endpoint, nil, bodyBuf, contentType) if err != nil { return err } @@ -244,63 +244,63 @@ func (f *functions) CreateFuncWithURL(funcConf *utils.FunctionConfig, pkgURL str } func (f *functions) StopFunction(tenant, namespace, name string) error { - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - return f.client.post(endpoint+"/stop", "") + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name) + return f.pulsar.Client.Post(endpoint+"/stop", "") } func (f *functions) StopFunctionWithID(tenant, namespace, name string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name, id) - return f.client.post(endpoint+"/stop", "") + return f.pulsar.Client.Post(endpoint+"/stop", "") } func (f *functions) DeleteFunction(tenant, namespace, name string) error { - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - return f.client.delete(endpoint) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name) + return f.pulsar.Client.Delete(endpoint) } func (f *functions) StartFunction(tenant, namespace, name string) error { - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - return f.client.post(endpoint+"/start", "") + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name) + return f.pulsar.Client.Post(endpoint+"/start", "") } func (f *functions) StartFunctionWithID(tenant, namespace, name string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name, id) - return f.client.post(endpoint+"/start", "") + return f.pulsar.Client.Post(endpoint+"/start", "") } func (f *functions) RestartFunction(tenant, namespace, name string) error { - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - return f.client.post(endpoint+"/restart", "") + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name) + return f.pulsar.Client.Post(endpoint+"/restart", "") } func (f *functions) RestartFunctionWithID(tenant, namespace, name string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name, id) - return f.client.post(endpoint+"/restart", "") + return f.pulsar.Client.Post(endpoint+"/restart", "") } func (f *functions) GetFunctions(tenant, namespace string) ([]string, error) { var functions []string - endpoint := f.client.endpoint(f.basePath, tenant, namespace) - err := f.client.get(endpoint, &functions) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace) + err := f.pulsar.Client.Get(endpoint, &functions) return functions, err } func (f *functions) GetFunction(tenant, namespace, name string) (utils.FunctionConfig, error) { var functionConfig utils.FunctionConfig - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - err := f.client.get(endpoint, &functionConfig) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name) + err := f.pulsar.Client.Get(endpoint, &functionConfig) return functionConfig, err } func (f *functions) UpdateFunction(functionConfig *utils.FunctionConfig, fileName string, updateOptions *utils.UpdateOptions) error { - endpoint := f.client.endpoint(f.basePath, functionConfig.Tenant, functionConfig.Namespace, functionConfig.Name) + endpoint := f.pulsar.endpoint(f.basePath, functionConfig.Tenant, functionConfig.Namespace, functionConfig.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -366,7 +366,7 @@ func (f *functions) UpdateFunction(functionConfig *utils.FunctionConfig, fileNam } contentType := multiPartWriter.FormDataContentType() - err = f.client.putWithMultiPart(endpoint, bodyBuf, contentType) + err = f.pulsar.Client.PutWithMultiPart(endpoint, bodyBuf, contentType) if err != nil { return err } @@ -376,7 +376,7 @@ func (f *functions) UpdateFunction(functionConfig *utils.FunctionConfig, fileNam func (f *functions) UpdateFunctionWithURL(functionConfig *utils.FunctionConfig, pkgURL string, updateOptions *utils.UpdateOptions) error { - endpoint := f.client.endpoint(f.basePath, functionConfig.Tenant, functionConfig.Namespace, functionConfig.Name) + endpoint := f.pulsar.endpoint(f.basePath, functionConfig.Tenant, functionConfig.Namespace, functionConfig.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -431,7 +431,7 @@ func (f *functions) UpdateFunctionWithURL(functionConfig *utils.FunctionConfig, } contentType := multiPartWriter.FormDataContentType() - err = f.client.putWithMultiPart(endpoint, bodyBuf, contentType) + err = f.pulsar.Client.PutWithMultiPart(endpoint, bodyBuf, contentType) if err != nil { return err } @@ -441,8 +441,8 @@ func (f *functions) UpdateFunctionWithURL(functionConfig *utils.FunctionConfig, func (f *functions) GetFunctionStatus(tenant, namespace, name string) (utils.FunctionStatus, error) { var functionStatus utils.FunctionStatus - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - err := f.client.get(endpoint+"/status", &functionStatus) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name) + err := f.pulsar.Client.Get(endpoint+"/status", &functionStatus) return functionStatus, err } @@ -450,15 +450,15 @@ func (f *functions) GetFunctionStatusWithInstanceID(tenant, namespace, name stri instanceID int) (utils.FunctionInstanceStatusData, error) { var functionInstanceStatusData utils.FunctionInstanceStatusData id := fmt.Sprintf("%d", instanceID) - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) - err := f.client.get(endpoint+"/status", &functionInstanceStatusData) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name, id) + err := f.pulsar.Client.Get(endpoint+"/status", &functionInstanceStatusData) return functionInstanceStatusData, err } func (f *functions) GetFunctionStats(tenant, namespace, name string) (utils.FunctionStats, error) { var functionStats utils.FunctionStats - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name) - err := f.client.get(endpoint+"/stats", &functionStats) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name) + err := f.pulsar.Client.Get(endpoint+"/stats", &functionStats) return functionStats, err } @@ -466,20 +466,20 @@ func (f *functions) GetFunctionStatsWithInstanceID(tenant, namespace, name strin instanceID int) (utils.FunctionInstanceStatsData, error) { var functionInstanceStatsData utils.FunctionInstanceStatsData id := fmt.Sprintf("%d", instanceID) - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, id) - err := f.client.get(endpoint+"/stats", &functionInstanceStatsData) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name, id) + err := f.pulsar.Client.Get(endpoint+"/stats", &functionInstanceStatsData) return functionInstanceStatsData, err } func (f *functions) GetFunctionState(tenant, namespace, name, key string) (utils.FunctionState, error) { var functionState utils.FunctionState - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, "state", key) - err := f.client.get(endpoint, &functionState) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name, "state", key) + err := f.pulsar.Client.Get(endpoint, &functionState) return functionState, err } func (f *functions) PutFunctionState(tenant, namespace, name string, state utils.FunctionState) error { - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, "state", state.Key) + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name, "state", state.Key) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -511,7 +511,7 @@ func (f *functions) PutFunctionState(tenant, namespace, name string, state utils contentType := multiPartWriter.FormDataContentType() - err = f.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) + err = f.pulsar.Client.PostWithMultiPart(endpoint, nil, bodyBuf, contentType) if err != nil { return err @@ -521,7 +521,7 @@ func (f *functions) PutFunctionState(tenant, namespace, name string, state utils } func (f *functions) TriggerFunction(tenant, namespace, name, topic, triggerValue, triggerFile string) (string, error) { - endpoint := f.client.endpoint(f.basePath, tenant, namespace, name, "trigger") + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name, "trigger") // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -580,7 +580,7 @@ func (f *functions) TriggerFunction(tenant, namespace, name, topic, triggerValue contentType := multiPartWriter.FormDataContentType() var str string - err := f.client.postWithMultiPart(endpoint, &str, bodyBuf, contentType) + err := f.pulsar.Client.PostWithMultiPart(endpoint, &str, bodyBuf, contentType) if err != nil { return "", err } diff --git a/pulsaradmin/pkg/pulsar/functions_worker.go b/pulsaradmin/pkg/pulsar/functions_worker.go index 8ce8a5eb3e..9703d157e8 100644 --- a/pulsaradmin/pkg/pulsar/functions_worker.go +++ b/pulsaradmin/pkg/pulsar/functions_worker.go @@ -39,23 +39,23 @@ type FunctionsWorker interface { } type worker struct { - client *client + pulsar *pulsarClient workerPath string workerStatsPath string } -func (c *client) FunctionsWorker() FunctionsWorker { +func (c *pulsarClient) FunctionsWorker() FunctionsWorker { return &worker{ - client: c, + pulsar: c, workerPath: "/worker", workerStatsPath: "/worker-stats", } } func (w *worker) GetFunctionsStats() ([]*utils.WorkerFunctionInstanceStats, error) { - endpoint := w.client.endpoint(w.workerStatsPath, "functionsmetrics") + endpoint := w.pulsar.endpoint(w.workerStatsPath, "functionsmetrics") var workerStats []*utils.WorkerFunctionInstanceStats - err := w.client.get(endpoint, &workerStats) + err := w.pulsar.Client.Get(endpoint, &workerStats) if err != nil { return nil, err } @@ -63,9 +63,9 @@ func (w *worker) GetFunctionsStats() ([]*utils.WorkerFunctionInstanceStats, erro } func (w *worker) GetMetrics() ([]*utils.Metrics, error) { - endpoint := w.client.endpoint(w.workerStatsPath, "metrics") + endpoint := w.pulsar.endpoint(w.workerStatsPath, "metrics") var metrics []*utils.Metrics - err := w.client.get(endpoint, &metrics) + err := w.pulsar.Client.Get(endpoint, &metrics) if err != nil { return nil, err } @@ -73,9 +73,9 @@ func (w *worker) GetMetrics() ([]*utils.Metrics, error) { } func (w *worker) GetCluster() ([]*utils.WorkerInfo, error) { - endpoint := w.client.endpoint(w.workerPath, "cluster") + endpoint := w.pulsar.endpoint(w.workerPath, "cluster") var workersInfo []*utils.WorkerInfo - err := w.client.get(endpoint, &workersInfo) + err := w.pulsar.Client.Get(endpoint, &workersInfo) if err != nil { return nil, err } @@ -83,9 +83,9 @@ func (w *worker) GetCluster() ([]*utils.WorkerInfo, error) { } func (w *worker) GetClusterLeader() (*utils.WorkerInfo, error) { - endpoint := w.client.endpoint(w.workerPath, "cluster", "leader") + endpoint := w.pulsar.endpoint(w.workerPath, "cluster", "leader") var workerInfo utils.WorkerInfo - err := w.client.get(endpoint, &workerInfo) + err := w.pulsar.Client.Get(endpoint, &workerInfo) if err != nil { return nil, err } @@ -93,9 +93,9 @@ func (w *worker) GetClusterLeader() (*utils.WorkerInfo, error) { } func (w *worker) GetAssignments() (map[string][]string, error) { - endpoint := w.client.endpoint(w.workerPath, "assignments") + endpoint := w.pulsar.endpoint(w.workerPath, "assignments") var assignments map[string][]string - err := w.client.get(endpoint, &assignments) + err := w.pulsar.Client.Get(endpoint, &assignments) if err != nil { return nil, err } diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index 02a91a9c5b..903617f869 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -250,22 +250,22 @@ type Namespaces interface { } type namespaces struct { - client *client + pulsar *pulsarClient basePath string } // Namespaces is used to access the namespaces endpoints -func (c *client) Namespaces() Namespaces { +func (c *pulsarClient) Namespaces() Namespaces { return &namespaces{ - client: c, + pulsar: c, basePath: "/namespaces", } } func (n *namespaces) GetNamespaces(tenant string) ([]string, error) { var namespaces []string - endpoint := n.client.endpoint(n.basePath, tenant) - err := n.client.get(endpoint, &namespaces) + endpoint := n.pulsar.endpoint(n.basePath, tenant) + err := n.pulsar.Client.Get(endpoint, &namespaces) return namespaces, err } @@ -275,8 +275,8 @@ func (n *namespaces) GetTopics(namespace string) ([]string, error) { if err != nil { return nil, err } - endpoint := n.client.endpoint(n.basePath, ns.String(), "topics") - err = n.client.get(endpoint, &topics) + endpoint := n.pulsar.endpoint(n.basePath, ns.String(), "topics") + err = n.pulsar.Client.Get(endpoint, &topics) return topics, err } @@ -286,8 +286,8 @@ func (n *namespaces) GetPolicies(namespace string) (*utils.Policies, error) { if err != nil { return nil, err } - endpoint := n.client.endpoint(n.basePath, ns.String()) - err = n.client.get(endpoint, &police) + endpoint := n.pulsar.endpoint(n.basePath, ns.String()) + err = n.pulsar.Client.Get(endpoint, &police) return &police, err } @@ -300,8 +300,8 @@ func (n *namespaces) CreateNsWithPolices(namespace string, policies utils.Polici if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, ns.String()) - return n.client.put(endpoint, &policies) + endpoint := n.pulsar.endpoint(n.basePath, ns.String()) + return n.pulsar.Client.Put(endpoint, &policies) } func (n *namespaces) CreateNsWithBundlesData(namespace string, bundleData *utils.BundlesData) error { @@ -309,11 +309,11 @@ func (n *namespaces) CreateNsWithBundlesData(namespace string, bundleData *utils if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, ns.String()) + endpoint := n.pulsar.endpoint(n.basePath, ns.String()) polices := new(utils.Policies) polices.Bundles = bundleData - return n.client.put(endpoint, &polices) + return n.pulsar.Client.Put(endpoint, &polices) } func (n *namespaces) CreateNamespace(namespace string) error { @@ -321,8 +321,8 @@ func (n *namespaces) CreateNamespace(namespace string) error { if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, ns.String()) - return n.client.put(endpoint, nil) + endpoint := n.pulsar.endpoint(n.basePath, ns.String()) + return n.pulsar.Client.Put(endpoint, nil) } func (n *namespaces) DeleteNamespace(namespace string) error { @@ -330,8 +330,8 @@ func (n *namespaces) DeleteNamespace(namespace string) error { if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, ns.String()) - return n.client.delete(endpoint) + endpoint := n.pulsar.endpoint(n.basePath, ns.String()) + return n.pulsar.Client.Delete(endpoint) } func (n *namespaces) DeleteNamespaceBundle(namespace string, bundleRange string) error { @@ -339,8 +339,8 @@ func (n *namespaces) DeleteNamespaceBundle(namespace string, bundleRange string) if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, ns.String(), bundleRange) - return n.client.delete(endpoint) + endpoint := n.pulsar.endpoint(n.basePath, ns.String(), bundleRange) + return n.pulsar.Client.Delete(endpoint) } func (n *namespaces) GetNamespaceMessageTTL(namespace string) (int, error) { @@ -349,8 +349,8 @@ func (n *namespaces) GetNamespaceMessageTTL(namespace string) (int, error) { if err != nil { return 0, err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "messageTTL") - err = n.client.get(endpoint, &ttl) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "messageTTL") + err = n.pulsar.Client.Get(endpoint, &ttl) return ttl, err } @@ -360,8 +360,8 @@ func (n *namespaces) SetNamespaceMessageTTL(namespace string, ttlInSeconds int) return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "messageTTL") - return n.client.post(endpoint, &ttlInSeconds) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "messageTTL") + return n.pulsar.Client.Post(endpoint, &ttlInSeconds) } func (n *namespaces) SetRetention(namespace string, policy utils.RetentionPolicies) error { @@ -369,8 +369,8 @@ func (n *namespaces) SetRetention(namespace string, policy utils.RetentionPolici if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "retention") - return n.client.post(endpoint, &policy) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "retention") + return n.pulsar.Client.Post(endpoint, &policy) } func (n *namespaces) GetRetention(namespace string) (*utils.RetentionPolicies, error) { @@ -379,8 +379,8 @@ func (n *namespaces) GetRetention(namespace string) (*utils.RetentionPolicies, e if err != nil { return nil, err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "retention") - err = n.client.get(endpoint, &policy) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "retention") + err = n.pulsar.Client.Get(endpoint, &policy) return &policy, err } @@ -390,8 +390,8 @@ func (n *namespaces) GetBacklogQuotaMap(namespace string) (map[utils.BacklogQuot if err != nil { return nil, err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuotaMap") - err = n.client.get(endpoint, &backlogQuotaMap) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "backlogQuotaMap") + err = n.pulsar.Client.Get(endpoint, &backlogQuotaMap) return backlogQuotaMap, err } @@ -400,8 +400,8 @@ func (n *namespaces) SetBacklogQuota(namespace string, backlogQuota utils.Backlo if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuota") - return n.client.post(endpoint, &backlogQuota) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "backlogQuota") + return n.pulsar.Client.Post(endpoint, &backlogQuota) } func (n *namespaces) RemoveBacklogQuota(namespace string) error { @@ -409,21 +409,21 @@ func (n *namespaces) RemoveBacklogQuota(namespace string) error { if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "backlogQuota") + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "backlogQuota") params := map[string]string{ "backlogQuotaType": string(utils.DestinationStorage), } - return n.client.deleteWithQueryParams(endpoint, nil, params) + return n.pulsar.Client.DeleteWithQueryParams(endpoint, params) } func (n *namespaces) SetSchemaValidationEnforced(namespace utils.NameSpaceName, schemaValidationEnforced bool) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaValidationEnforced") - return n.client.post(endpoint, schemaValidationEnforced) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "schemaValidationEnforced") + return n.pulsar.Client.Post(endpoint, schemaValidationEnforced) } func (n *namespaces) GetSchemaValidationEnforced(namespace utils.NameSpaceName) (bool, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaValidationEnforced") - r, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "schemaValidationEnforced") + r, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) if err != nil { return false, err } @@ -432,15 +432,15 @@ func (n *namespaces) GetSchemaValidationEnforced(namespace utils.NameSpaceName) func (n *namespaces) SetSchemaAutoUpdateCompatibilityStrategy(namespace utils.NameSpaceName, strategy utils.SchemaCompatibilityStrategy) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaAutoUpdateCompatibilityStrategy") - return n.client.put(endpoint, strategy.String()) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "schemaAutoUpdateCompatibilityStrategy") + return n.pulsar.Client.Put(endpoint, strategy.String()) } func (n *namespaces) GetSchemaAutoUpdateCompatibilityStrategy(namespace utils.NameSpaceName) ( utils.SchemaCompatibilityStrategy, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "schemaAutoUpdateCompatibilityStrategy") - b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "schemaAutoUpdateCompatibilityStrategy") + b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) if err != nil { return "", err } @@ -452,18 +452,18 @@ func (n *namespaces) GetSchemaAutoUpdateCompatibilityStrategy(namespace utils.Na } func (n *namespaces) ClearOffloadDeleteLag(namespace utils.NameSpaceName) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") - return n.client.delete(endpoint) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") + return n.pulsar.Client.Delete(endpoint) } func (n *namespaces) SetOffloadDeleteLag(namespace utils.NameSpaceName, timeMs int64) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") - return n.client.put(endpoint, timeMs) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") + return n.pulsar.Client.Put(endpoint, timeMs) } func (n *namespaces) GetOffloadDeleteLag(namespace utils.NameSpaceName) (int64, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") - b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") + b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) if err != nil { return -1, err } @@ -471,13 +471,13 @@ func (n *namespaces) GetOffloadDeleteLag(namespace utils.NameSpaceName) (int64, } func (n *namespaces) SetMaxConsumersPerSubscription(namespace utils.NameSpaceName, max int) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerSubscription") - return n.client.post(endpoint, max) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "maxConsumersPerSubscription") + return n.pulsar.Client.Post(endpoint, max) } func (n *namespaces) GetMaxConsumersPerSubscription(namespace utils.NameSpaceName) (int, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerSubscription") - b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "maxConsumersPerSubscription") + b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) if err != nil { return -1, err } @@ -485,13 +485,13 @@ func (n *namespaces) GetMaxConsumersPerSubscription(namespace utils.NameSpaceNam } func (n *namespaces) SetOffloadThreshold(namespace utils.NameSpaceName, threshold int64) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadThreshold") - return n.client.put(endpoint, threshold) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "offloadThreshold") + return n.pulsar.Client.Put(endpoint, threshold) } func (n *namespaces) GetOffloadThreshold(namespace utils.NameSpaceName) (int64, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "offloadThreshold") - b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "offloadThreshold") + b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) if err != nil { return -1, err } @@ -499,13 +499,13 @@ func (n *namespaces) GetOffloadThreshold(namespace utils.NameSpaceName) (int64, } func (n *namespaces) SetMaxConsumersPerTopic(namespace utils.NameSpaceName, max int) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerTopic") - return n.client.post(endpoint, max) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "maxConsumersPerTopic") + return n.pulsar.Client.Post(endpoint, max) } func (n *namespaces) GetMaxConsumersPerTopic(namespace utils.NameSpaceName) (int, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxConsumersPerTopic") - b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "maxConsumersPerTopic") + b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) if err != nil { return -1, err } @@ -513,13 +513,13 @@ func (n *namespaces) GetMaxConsumersPerTopic(namespace utils.NameSpaceName) (int } func (n *namespaces) SetCompactionThreshold(namespace utils.NameSpaceName, threshold int64) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "compactionThreshold") - return n.client.put(endpoint, threshold) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "compactionThreshold") + return n.pulsar.Client.Put(endpoint, threshold) } func (n *namespaces) GetCompactionThreshold(namespace utils.NameSpaceName) (int64, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "compactionThreshold") - b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "compactionThreshold") + b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) if err != nil { return -1, err } @@ -527,13 +527,13 @@ func (n *namespaces) GetCompactionThreshold(namespace utils.NameSpaceName) (int6 } func (n *namespaces) SetMaxProducersPerTopic(namespace utils.NameSpaceName, max int) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxProducersPerTopic") - return n.client.post(endpoint, max) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "maxProducersPerTopic") + return n.pulsar.Client.Post(endpoint, max) } func (n *namespaces) GetMaxProducersPerTopic(namespace utils.NameSpaceName) (int, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "maxProducersPerTopic") - b, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "maxProducersPerTopic") + b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) if err != nil { return -1, err } @@ -546,8 +546,8 @@ func (n *namespaces) GetNamespaceReplicationClusters(namespace string) ([]string if err != nil { return nil, err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "replication") - err = n.client.get(endpoint, &data) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "replication") + err = n.pulsar.Client.Get(endpoint, &data) return data, err } @@ -556,8 +556,8 @@ func (n *namespaces) SetNamespaceReplicationClusters(namespace string, clusterId if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "replication") - return n.client.post(endpoint, &clusterIds) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "replication") + return n.pulsar.Client.Post(endpoint, &clusterIds) } func (n *namespaces) SetNamespaceAntiAffinityGroup(namespace string, namespaceAntiAffinityGroup string) error { @@ -565,17 +565,17 @@ func (n *namespaces) SetNamespaceAntiAffinityGroup(namespace string, namespaceAn if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "antiAffinity") - return n.client.post(endpoint, namespaceAntiAffinityGroup) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "antiAffinity") + return n.pulsar.Client.Post(endpoint, namespaceAntiAffinityGroup) } func (n *namespaces) GetAntiAffinityNamespaces(tenant, cluster, namespaceAntiAffinityGroup string) ([]string, error) { var data []string - endpoint := n.client.endpoint(n.basePath, cluster, "antiAffinity", namespaceAntiAffinityGroup) + endpoint := n.pulsar.endpoint(n.basePath, cluster, "antiAffinity", namespaceAntiAffinityGroup) params := map[string]string{ "property": tenant, } - _, err := n.client.getWithQueryParams(endpoint, &data, params, false) + _, err := n.pulsar.Client.GetWithQueryParams(endpoint, &data, params, false) return data, err } @@ -584,8 +584,8 @@ func (n *namespaces) GetNamespaceAntiAffinityGroup(namespace string) (string, er if err != nil { return "", err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "antiAffinity") - data, err := n.client.getWithQueryParams(endpoint, nil, nil, false) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "antiAffinity") + data, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) return string(data), err } @@ -594,8 +594,8 @@ func (n *namespaces) DeleteNamespaceAntiAffinityGroup(namespace string) error { if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "antiAffinity") - return n.client.delete(endpoint) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "antiAffinity") + return n.pulsar.Client.Delete(endpoint) } func (n *namespaces) SetDeduplicationStatus(namespace string, enableDeduplication bool) error { @@ -603,8 +603,8 @@ func (n *namespaces) SetDeduplicationStatus(namespace string, enableDeduplicatio if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "deduplication") - return n.client.post(endpoint, enableDeduplication) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "deduplication") + return n.pulsar.Client.Post(endpoint, enableDeduplication) } func (n *namespaces) SetPersistence(namespace string, persistence utils.PersistencePolicies) error { @@ -612,8 +612,8 @@ func (n *namespaces) SetPersistence(namespace string, persistence utils.Persiste if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence") - return n.client.post(endpoint, &persistence) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "persistence") + return n.pulsar.Client.Post(endpoint, &persistence) } func (n *namespaces) SetBookieAffinityGroup(namespace string, bookieAffinityGroup utils.BookieAffinityGroupData) error { @@ -621,8 +621,8 @@ func (n *namespaces) SetBookieAffinityGroup(namespace string, bookieAffinityGrou if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence", "bookieAffinity") - return n.client.post(endpoint, &bookieAffinityGroup) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "persistence", "bookieAffinity") + return n.pulsar.Client.Post(endpoint, &bookieAffinityGroup) } func (n *namespaces) DeleteBookieAffinityGroup(namespace string) error { @@ -630,8 +630,8 @@ func (n *namespaces) DeleteBookieAffinityGroup(namespace string) error { if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence", "bookieAffinity") - return n.client.delete(endpoint) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "persistence", "bookieAffinity") + return n.pulsar.Client.Delete(endpoint) } func (n *namespaces) GetBookieAffinityGroup(namespace string) (*utils.BookieAffinityGroupData, error) { @@ -640,8 +640,8 @@ func (n *namespaces) GetBookieAffinityGroup(namespace string) (*utils.BookieAffi if err != nil { return nil, err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence", "bookieAffinity") - err = n.client.get(endpoint, &data) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "persistence", "bookieAffinity") + err = n.pulsar.Client.Get(endpoint, &data) return &data, err } @@ -651,8 +651,8 @@ func (n *namespaces) GetPersistence(namespace string) (*utils.PersistencePolicie if err != nil { return nil, err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "persistence") - err = n.client.get(endpoint, &persistence) + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "persistence") + err = n.pulsar.Client.Get(endpoint, &persistence) return &persistence, err } @@ -661,8 +661,8 @@ func (n *namespaces) Unload(namespace string) error { if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), "unload") - return n.client.put(endpoint, "") + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "unload") + return n.pulsar.Client.Put(endpoint, "") } func (n *namespaces) UnloadNamespaceBundle(namespace, bundle string) error { @@ -670,8 +670,8 @@ func (n *namespaces) UnloadNamespaceBundle(namespace, bundle string) error { if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), bundle, "unload") - return n.client.put(endpoint, "") + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), bundle, "unload") + return n.pulsar.Client.Put(endpoint, "") } func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitBundles bool) error { @@ -679,132 +679,132 @@ func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitB if err != nil { return err } - endpoint := n.client.endpoint(n.basePath, nsName.String(), bundle, "split") + endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), bundle, "split") params := map[string]string{ "unload": strconv.FormatBool(unloadSplitBundles), } - return n.client.putWithQueryParams(endpoint, "", nil, params) + return n.pulsar.Client.PutWithQueryParams(endpoint, "", nil, params) } func (n *namespaces) GetNamespacePermissions(namespace utils.NameSpaceName) (map[string][]common.AuthAction, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions") + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "permissions") var permissions map[string][]common.AuthAction - err := n.client.get(endpoint, &permissions) + err := n.pulsar.Client.Get(endpoint, &permissions) return permissions, err } func (n *namespaces) GrantNamespacePermission(namespace utils.NameSpaceName, role string, action []common.AuthAction) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", role) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "permissions", role) s := make([]string, 0) for _, v := range action { s = append(s, v.String()) } - return n.client.post(endpoint, s) + return n.pulsar.Client.Post(endpoint, s) } func (n *namespaces) RevokeNamespacePermission(namespace utils.NameSpaceName, role string) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", role) - return n.client.delete(endpoint) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "permissions", role) + return n.pulsar.Client.Delete(endpoint) } func (n *namespaces) GrantSubPermission(namespace utils.NameSpaceName, sName string, roles []string) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "permissions", "subscription", sName) - return n.client.post(endpoint, roles) + return n.pulsar.Client.Post(endpoint, roles) } func (n *namespaces) RevokeSubPermission(namespace utils.NameSpaceName, sName, role string) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "permissions", + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "permissions", "subscription", sName, role) - return n.client.delete(endpoint) + return n.pulsar.Client.Delete(endpoint) } func (n *namespaces) SetSubscriptionAuthMode(namespace utils.NameSpaceName, mode utils.SubscriptionAuthMode) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionAuthMode") - return n.client.post(endpoint, mode.String()) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "subscriptionAuthMode") + return n.pulsar.Client.Post(endpoint, mode.String()) } func (n *namespaces) SetEncryptionRequiredStatus(namespace utils.NameSpaceName, encrypt bool) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "encryptionRequired") - return n.client.post(endpoint, strconv.FormatBool(encrypt)) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "encryptionRequired") + return n.pulsar.Client.Post(endpoint, strconv.FormatBool(encrypt)) } func (n *namespaces) UnsubscribeNamespace(namespace utils.NameSpaceName, sName string) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "unsubscribe", url.QueryEscape(sName)) - return n.client.post(endpoint, "") + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "unsubscribe", url.QueryEscape(sName)) + return n.pulsar.Client.Post(endpoint, "") } func (n *namespaces) UnsubscribeNamespaceBundle(namespace utils.NameSpaceName, bundle, sName string) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "unsubscribe", url.QueryEscape(sName)) - return n.client.post(endpoint, "") + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), bundle, "unsubscribe", url.QueryEscape(sName)) + return n.pulsar.Client.Post(endpoint, "") } func (n *namespaces) ClearNamespaceBundleBacklogForSubscription(namespace utils.NameSpaceName, bundle, sName string) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog", url.QueryEscape(sName)) - return n.client.post(endpoint, "") + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog", url.QueryEscape(sName)) + return n.pulsar.Client.Post(endpoint, "") } func (n *namespaces) ClearNamespaceBundleBacklog(namespace utils.NameSpaceName, bundle string) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog") - return n.client.post(endpoint, "") + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog") + return n.pulsar.Client.Post(endpoint, "") } func (n *namespaces) ClearNamespaceBacklogForSubscription(namespace utils.NameSpaceName, sName string) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "clearBacklog", url.QueryEscape(sName)) - return n.client.post(endpoint, "") + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "clearBacklog", url.QueryEscape(sName)) + return n.pulsar.Client.Post(endpoint, "") } func (n *namespaces) ClearNamespaceBacklog(namespace utils.NameSpaceName) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "clearBacklog") - return n.client.post(endpoint, "") + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "clearBacklog") + return n.pulsar.Client.Post(endpoint, "") } func (n *namespaces) SetReplicatorDispatchRate(namespace utils.NameSpaceName, rate utils.DispatchRate) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "replicatorDispatchRate") - return n.client.post(endpoint, rate) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "replicatorDispatchRate") + return n.pulsar.Client.Post(endpoint, rate) } func (n *namespaces) GetReplicatorDispatchRate(namespace utils.NameSpaceName) (utils.DispatchRate, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "replicatorDispatchRate") + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "replicatorDispatchRate") var rate utils.DispatchRate - err := n.client.get(endpoint, &rate) + err := n.pulsar.Client.Get(endpoint, &rate) return rate, err } func (n *namespaces) SetSubscriptionDispatchRate(namespace utils.NameSpaceName, rate utils.DispatchRate) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionDispatchRate") - return n.client.post(endpoint, rate) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "subscriptionDispatchRate") + return n.pulsar.Client.Post(endpoint, rate) } func (n *namespaces) GetSubscriptionDispatchRate(namespace utils.NameSpaceName) (utils.DispatchRate, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscriptionDispatchRate") + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "subscriptionDispatchRate") var rate utils.DispatchRate - err := n.client.get(endpoint, &rate) + err := n.pulsar.Client.Get(endpoint, &rate) return rate, err } func (n *namespaces) SetSubscribeRate(namespace utils.NameSpaceName, rate utils.SubscribeRate) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscribeRate") - return n.client.post(endpoint, rate) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "subscribeRate") + return n.pulsar.Client.Post(endpoint, rate) } func (n *namespaces) GetSubscribeRate(namespace utils.NameSpaceName) (utils.SubscribeRate, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "subscribeRate") + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "subscribeRate") var rate utils.SubscribeRate - err := n.client.get(endpoint, &rate) + err := n.pulsar.Client.Get(endpoint, &rate) return rate, err } func (n *namespaces) SetDispatchRate(namespace utils.NameSpaceName, rate utils.DispatchRate) error { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "dispatchRate") - return n.client.post(endpoint, rate) + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "dispatchRate") + return n.pulsar.Client.Post(endpoint, rate) } func (n *namespaces) GetDispatchRate(namespace utils.NameSpaceName) (utils.DispatchRate, error) { - endpoint := n.client.endpoint(n.basePath, namespace.String(), "dispatchRate") + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "dispatchRate") var rate utils.DispatchRate - err := n.client.get(endpoint, &rate) + err := n.pulsar.Client.Get(endpoint, &rate) return rate, err } diff --git a/pulsaradmin/pkg/pulsar/ns_isolation_policy.go b/pulsaradmin/pkg/pulsar/ns_isolation_policy.go index faaf70554d..0ad8b03e18 100644 --- a/pulsaradmin/pkg/pulsar/ns_isolation_policy.go +++ b/pulsaradmin/pkg/pulsar/ns_isolation_policy.go @@ -42,13 +42,13 @@ type NsIsolationPolicy interface { } type nsIsolationPolicy struct { - client *client + pulsar *pulsarClient basePath string } -func (c *client) NsIsolationPolicy() NsIsolationPolicy { +func (c *pulsarClient) NsIsolationPolicy() NsIsolationPolicy { return &nsIsolationPolicy{ - client: c, + pulsar: c, basePath: "/clusters", } } @@ -60,20 +60,20 @@ func (n *nsIsolationPolicy) CreateNamespaceIsolationPolicy(cluster, policyName s func (n *nsIsolationPolicy) setNamespaceIsolationPolicy(cluster, policyName string, namespaceIsolationData utils.NamespaceIsolationData) error { - endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", policyName) - return n.client.post(endpoint, &namespaceIsolationData) + endpoint := n.pulsar.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", policyName) + return n.pulsar.Client.Post(endpoint, &namespaceIsolationData) } func (n *nsIsolationPolicy) DeleteNamespaceIsolationPolicy(cluster, policyName string) error { - endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", policyName) - return n.client.delete(endpoint) + endpoint := n.pulsar.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", policyName) + return n.pulsar.Client.Delete(endpoint) } func (n *nsIsolationPolicy) GetNamespaceIsolationPolicy(cluster, policyName string) ( *utils.NamespaceIsolationData, error) { - endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", policyName) + endpoint := n.pulsar.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", policyName) var nsIsolationData utils.NamespaceIsolationData - err := n.client.get(endpoint, &nsIsolationData) + err := n.pulsar.Client.Get(endpoint, &nsIsolationData) if err != nil { return nil, err } @@ -82,9 +82,9 @@ func (n *nsIsolationPolicy) GetNamespaceIsolationPolicy(cluster, policyName stri func (n *nsIsolationPolicy) GetNamespaceIsolationPolicies(cluster string) ( map[string]utils.NamespaceIsolationData, error) { - endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies") + endpoint := n.pulsar.endpoint(n.basePath, cluster, "namespaceIsolationPolicies") var tmpMap map[string]utils.NamespaceIsolationData - err := n.client.get(endpoint, &tmpMap) + err := n.pulsar.Client.Get(endpoint, &tmpMap) if err != nil { return nil, err } @@ -93,9 +93,9 @@ func (n *nsIsolationPolicy) GetNamespaceIsolationPolicies(cluster string) ( func (n *nsIsolationPolicy) GetBrokersWithNamespaceIsolationPolicy(cluster string) ( []utils.BrokerNamespaceIsolationData, error) { - endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", "brokers") + endpoint := n.pulsar.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", "brokers") var res []utils.BrokerNamespaceIsolationData - err := n.client.get(endpoint, &res) + err := n.pulsar.Client.Get(endpoint, &res) if err != nil { return nil, err } @@ -104,9 +104,9 @@ func (n *nsIsolationPolicy) GetBrokersWithNamespaceIsolationPolicy(cluster strin func (n *nsIsolationPolicy) GetBrokerWithNamespaceIsolationPolicy(cluster, broker string) (*utils.BrokerNamespaceIsolationData, error) { - endpoint := n.client.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", "brokers", broker) + endpoint := n.pulsar.endpoint(n.basePath, cluster, "namespaceIsolationPolicies", "brokers", broker) var brokerNamespaceIsolationData utils.BrokerNamespaceIsolationData - err := n.client.get(endpoint, &brokerNamespaceIsolationData) + err := n.pulsar.Client.Get(endpoint, &brokerNamespaceIsolationData) if err != nil { return nil, err } diff --git a/pulsaradmin/pkg/pulsar/resource_quotas.go b/pulsaradmin/pkg/pulsar/resource_quotas.go index 50f2bd3685..96e856f2f2 100644 --- a/pulsaradmin/pkg/pulsar/resource_quotas.go +++ b/pulsaradmin/pkg/pulsar/resource_quotas.go @@ -39,21 +39,21 @@ type ResourceQuotas interface { } type resource struct { - client *client + pulsar *pulsarClient basePath string } -func (c *client) ResourceQuotas() ResourceQuotas { +func (c *pulsarClient) ResourceQuotas() ResourceQuotas { return &resource{ - client: c, + pulsar: c, basePath: "/resource-quotas", } } func (r *resource) GetDefaultResourceQuota() (*utils.ResourceQuota, error) { - endpoint := r.client.endpoint(r.basePath) + endpoint := r.pulsar.endpoint(r.basePath) var quota utils.ResourceQuota - err := r.client.get(endpoint, "a) + err := r.pulsar.Client.Get(endpoint, "a) if err != nil { return nil, err } @@ -61,14 +61,14 @@ func (r *resource) GetDefaultResourceQuota() (*utils.ResourceQuota, error) { } func (r *resource) SetDefaultResourceQuota(quota utils.ResourceQuota) error { - endpoint := r.client.endpoint(r.basePath) - return r.client.post(endpoint, "a) + endpoint := r.pulsar.endpoint(r.basePath) + return r.pulsar.Client.Post(endpoint, "a) } func (r *resource) GetNamespaceBundleResourceQuota(namespace, bundle string) (*utils.ResourceQuota, error) { - endpoint := r.client.endpoint(r.basePath, namespace, bundle) + endpoint := r.pulsar.endpoint(r.basePath, namespace, bundle) var quota utils.ResourceQuota - err := r.client.get(endpoint, "a) + err := r.pulsar.Client.Get(endpoint, "a) if err != nil { return nil, err } @@ -76,11 +76,11 @@ func (r *resource) GetNamespaceBundleResourceQuota(namespace, bundle string) (*u } func (r *resource) SetNamespaceBundleResourceQuota(namespace, bundle string, quota utils.ResourceQuota) error { - endpoint := r.client.endpoint(r.basePath, namespace, bundle) - return r.client.post(endpoint, "a) + endpoint := r.pulsar.endpoint(r.basePath, namespace, bundle) + return r.pulsar.Client.Post(endpoint, "a) } func (r *resource) ResetNamespaceBundleResourceQuota(namespace, bundle string) error { - endpoint := r.client.endpoint(r.basePath, namespace, bundle) - return r.client.delete(endpoint) + endpoint := r.pulsar.endpoint(r.basePath, namespace, bundle) + return r.pulsar.Client.Delete(endpoint) } diff --git a/pulsaradmin/pkg/pulsar/schema.go b/pulsaradmin/pkg/pulsar/schema.go index 55e4e10bb5..5769bc2abb 100644 --- a/pulsaradmin/pkg/pulsar/schema.go +++ b/pulsaradmin/pkg/pulsar/schema.go @@ -43,14 +43,14 @@ type Schema interface { } type schemas struct { - client *client + pulsar *pulsarClient basePath string } // Schemas is used to access the schemas endpoints -func (c *client) Schemas() Schema { +func (c *pulsarClient) Schemas() Schema { return &schemas{ - client: c, + pulsar: c, basePath: "/schemas", } } @@ -61,10 +61,10 @@ func (s *schemas) GetSchemaInfo(topic string) (*utils.SchemaInfo, error) { return nil, err } var response utils.GetSchemaResponse - endpoint := s.client.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), + endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetEncodedTopic(), "schema") - err = s.client.get(endpoint, &response) + err = s.pulsar.Client.Get(endpoint, &response) if err != nil { return nil, err } @@ -79,10 +79,10 @@ func (s *schemas) GetSchemaInfoWithVersion(topic string) (*utils.SchemaInfoWithV return nil, err } var response utils.GetSchemaResponse - endpoint := s.client.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), + endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetEncodedTopic(), "schema") - err = s.client.get(endpoint, &response) + err = s.pulsar.Client.Get(endpoint, &response) if err != nil { fmt.Println("err:", err.Error()) return nil, err @@ -99,10 +99,10 @@ func (s *schemas) GetSchemaInfoByVersion(topic string, version int64) (*utils.Sc } var response utils.GetSchemaResponse - endpoint := s.client.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetEncodedTopic(), + endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetEncodedTopic(), "schema", strconv.FormatInt(version, 10)) - err = s.client.get(endpoint, &response) + err = s.pulsar.Client.Get(endpoint, &response) if err != nil { return nil, err } @@ -117,12 +117,12 @@ func (s *schemas) DeleteSchema(topic string) error { return err } - endpoint := s.client.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), + endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetEncodedTopic(), "schema") fmt.Println(endpoint) - return s.client.delete(endpoint) + return s.pulsar.Client.Delete(endpoint) } func (s *schemas) CreateSchemaByPayload(topic string, schemaPayload utils.PostSchemaPayload) error { @@ -131,8 +131,8 @@ func (s *schemas) CreateSchemaByPayload(topic string, schemaPayload utils.PostSc return err } - endpoint := s.client.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), + endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetEncodedTopic(), "schema") - return s.client.post(endpoint, &schemaPayload) + return s.pulsar.Client.Post(endpoint, &schemaPayload) } diff --git a/pulsaradmin/pkg/pulsar/sinks.go b/pulsaradmin/pkg/pulsar/sinks.go index c9033cec5e..295266b49a 100644 --- a/pulsaradmin/pkg/pulsar/sinks.go +++ b/pulsaradmin/pkg/pulsar/sinks.go @@ -86,14 +86,14 @@ type Sinks interface { } type sinks struct { - client *client + pulsar *pulsarClient basePath string } // Sinks is used to access the sinks endpoints -func (c *client) Sinks() Sinks { +func (c *pulsarClient) Sinks() Sinks { return &sinks{ - client: c, + pulsar: c, basePath: "/sinks", } } @@ -114,20 +114,20 @@ func (s *sinks) createTextFromFiled(w *multipart.Writer, value string) (io.Write func (s *sinks) ListSinks(tenant, namespace string) ([]string, error) { var sinks []string - endpoint := s.client.endpoint(s.basePath, tenant, namespace) - err := s.client.get(endpoint, &sinks) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace) + err := s.pulsar.Client.Get(endpoint, &sinks) return sinks, err } func (s *sinks) GetSink(tenant, namespace, sink string) (utils.SinkConfig, error) { var sinkConfig utils.SinkConfig - endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) - err := s.client.get(endpoint, &sinkConfig) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink) + err := s.pulsar.Client.Get(endpoint, &sinkConfig) return sinkConfig, err } func (s *sinks) CreateSink(config *utils.SinkConfig, fileName string) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + endpoint := s.pulsar.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -176,7 +176,7 @@ func (s *sinks) CreateSink(config *utils.SinkConfig, fileName string) error { } contentType := multiPartWriter.FormDataContentType() - err = s.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) + err = s.pulsar.Client.PostWithMultiPart(endpoint, nil, bodyBuf, contentType) if err != nil { return err } @@ -185,7 +185,7 @@ func (s *sinks) CreateSink(config *utils.SinkConfig, fileName string) error { } func (s *sinks) CreateSinkWithURL(config *utils.SinkConfig, pkgURL string) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + endpoint := s.pulsar.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -221,7 +221,7 @@ func (s *sinks) CreateSinkWithURL(config *utils.SinkConfig, pkgURL string) error } contentType := multiPartWriter.FormDataContentType() - err = s.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) + err = s.pulsar.Client.PostWithMultiPart(endpoint, nil, bodyBuf, contentType) if err != nil { return err } @@ -230,7 +230,7 @@ func (s *sinks) CreateSinkWithURL(config *utils.SinkConfig, pkgURL string) error } func (s *sinks) UpdateSink(config *utils.SinkConfig, fileName string, updateOptions *utils.UpdateOptions) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + endpoint := s.pulsar.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -296,7 +296,7 @@ func (s *sinks) UpdateSink(config *utils.SinkConfig, fileName string, updateOpti } contentType := multiPartWriter.FormDataContentType() - err = s.client.putWithMultiPart(endpoint, bodyBuf, contentType) + err = s.pulsar.Client.PutWithMultiPart(endpoint, bodyBuf, contentType) if err != nil { return err } @@ -305,7 +305,7 @@ func (s *sinks) UpdateSink(config *utils.SinkConfig, fileName string, updateOpti } func (s *sinks) UpdateSinkWithURL(config *utils.SinkConfig, pkgURL string, updateOptions *utils.UpdateOptions) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + endpoint := s.pulsar.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -360,7 +360,7 @@ func (s *sinks) UpdateSinkWithURL(config *utils.SinkConfig, pkgURL string, updat } contentType := multiPartWriter.FormDataContentType() - err = s.client.putWithMultiPart(endpoint, bodyBuf, contentType) + err = s.pulsar.Client.PutWithMultiPart(endpoint, bodyBuf, contentType) if err != nil { return err } @@ -369,69 +369,69 @@ func (s *sinks) UpdateSinkWithURL(config *utils.SinkConfig, pkgURL string, updat } func (s *sinks) DeleteSink(tenant, namespace, sink string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) - return s.client.delete(endpoint) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink) + return s.pulsar.Client.Delete(endpoint) } func (s *sinks) GetSinkStatus(tenant, namespace, sink string) (utils.SinkStatus, error) { var sinkStatus utils.SinkStatus - endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) - err := s.client.get(endpoint+"/status", &sinkStatus) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink) + err := s.pulsar.Client.Get(endpoint+"/status", &sinkStatus) return sinkStatus, err } func (s *sinks) GetSinkStatusWithID(tenant, namespace, sink string, id int) (utils.SinkInstanceStatusData, error) { var sinkInstanceStatusData utils.SinkInstanceStatusData instanceID := fmt.Sprintf("%d", id) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink, instanceID) - err := s.client.get(endpoint+"/status", &sinkInstanceStatusData) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink, instanceID) + err := s.pulsar.Client.Get(endpoint+"/status", &sinkInstanceStatusData) return sinkInstanceStatusData, err } func (s *sinks) RestartSink(tenant, namespace, sink string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) - return s.client.post(endpoint+"/restart", "") + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink) + return s.pulsar.Client.Post(endpoint+"/restart", "") } func (s *sinks) RestartSinkWithID(tenant, namespace, sink string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink, id) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink, id) - return s.client.post(endpoint+"/restart", "") + return s.pulsar.Client.Post(endpoint+"/restart", "") } func (s *sinks) StopSink(tenant, namespace, sink string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) - return s.client.post(endpoint+"/stop", "") + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink) + return s.pulsar.Client.Post(endpoint+"/stop", "") } func (s *sinks) StopSinkWithID(tenant, namespace, sink string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink, id) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink, id) - return s.client.post(endpoint+"/stop", "") + return s.pulsar.Client.Post(endpoint+"/stop", "") } func (s *sinks) StartSink(tenant, namespace, sink string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink) - return s.client.post(endpoint+"/start", "") + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink) + return s.pulsar.Client.Post(endpoint+"/start", "") } func (s *sinks) StartSinkWithID(tenant, namespace, sink string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, sink, id) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink, id) - return s.client.post(endpoint+"/start", "") + return s.pulsar.Client.Post(endpoint+"/start", "") } func (s *sinks) GetBuiltInSinks() ([]*utils.ConnectorDefinition, error) { var connectorDefinition []*utils.ConnectorDefinition - endpoint := s.client.endpoint(s.basePath, "builtinSinks") - err := s.client.get(endpoint, &connectorDefinition) + endpoint := s.pulsar.endpoint(s.basePath, "builtinSinks") + err := s.pulsar.Client.Get(endpoint, &connectorDefinition) return connectorDefinition, err } func (s *sinks) ReloadBuiltInSinks() error { - endpoint := s.client.endpoint(s.basePath, "reloadBuiltInSinks") - return s.client.post(endpoint, "") + endpoint := s.pulsar.endpoint(s.basePath, "reloadBuiltInSinks") + return s.pulsar.Client.Post(endpoint, "") } diff --git a/pulsaradmin/pkg/pulsar/sources.go b/pulsaradmin/pkg/pulsar/sources.go index 1c5cac075a..fd2d29a854 100644 --- a/pulsaradmin/pkg/pulsar/sources.go +++ b/pulsaradmin/pkg/pulsar/sources.go @@ -87,14 +87,14 @@ type Sources interface { } type sources struct { - client *client + pulsar *pulsarClient basePath string } // Sources is used to access the sources endpoints -func (c *client) Sources() Sources { +func (c *pulsarClient) Sources() Sources { return &sources{ - client: c, + pulsar: c, basePath: "/sources", } } @@ -115,20 +115,20 @@ func (s *sources) createTextFromFiled(w *multipart.Writer, value string) (io.Wri func (s *sources) ListSources(tenant, namespace string) ([]string, error) { var sources []string - endpoint := s.client.endpoint(s.basePath, tenant, namespace) - err := s.client.get(endpoint, &sources) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace) + err := s.pulsar.Client.Get(endpoint, &sources) return sources, err } func (s *sources) GetSource(tenant, namespace, source string) (utils.SourceConfig, error) { var sourceConfig utils.SourceConfig - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - err := s.client.get(endpoint, &sourceConfig) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source) + err := s.pulsar.Client.Get(endpoint, &sourceConfig) return sourceConfig, err } func (s *sources) CreateSource(config *utils.SourceConfig, fileName string) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + endpoint := s.pulsar.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -177,7 +177,7 @@ func (s *sources) CreateSource(config *utils.SourceConfig, fileName string) erro } contentType := multiPartWriter.FormDataContentType() - err = s.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) + err = s.pulsar.Client.PostWithMultiPart(endpoint, nil, bodyBuf, contentType) if err != nil { return err } @@ -186,7 +186,7 @@ func (s *sources) CreateSource(config *utils.SourceConfig, fileName string) erro } func (s *sources) CreateSourceWithURL(config *utils.SourceConfig, pkgURL string) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + endpoint := s.pulsar.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -222,7 +222,7 @@ func (s *sources) CreateSourceWithURL(config *utils.SourceConfig, pkgURL string) } contentType := multiPartWriter.FormDataContentType() - err = s.client.postWithMultiPart(endpoint, nil, bodyBuf, contentType) + err = s.pulsar.Client.PostWithMultiPart(endpoint, nil, bodyBuf, contentType) if err != nil { return err } @@ -231,7 +231,7 @@ func (s *sources) CreateSourceWithURL(config *utils.SourceConfig, pkgURL string) } func (s *sources) UpdateSource(config *utils.SourceConfig, fileName string, updateOptions *utils.UpdateOptions) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + endpoint := s.pulsar.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -297,7 +297,7 @@ func (s *sources) UpdateSource(config *utils.SourceConfig, fileName string, upda } contentType := multiPartWriter.FormDataContentType() - err = s.client.putWithMultiPart(endpoint, bodyBuf, contentType) + err = s.pulsar.Client.PutWithMultiPart(endpoint, bodyBuf, contentType) if err != nil { return err } @@ -307,7 +307,7 @@ func (s *sources) UpdateSource(config *utils.SourceConfig, fileName string, upda func (s *sources) UpdateSourceWithURL(config *utils.SourceConfig, pkgURL string, updateOptions *utils.UpdateOptions) error { - endpoint := s.client.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) + endpoint := s.pulsar.endpoint(s.basePath, config.Tenant, config.Namespace, config.Name) // buffer to store our request as bytes bodyBuf := bytes.NewBufferString("") @@ -362,7 +362,7 @@ func (s *sources) UpdateSourceWithURL(config *utils.SourceConfig, pkgURL string, } contentType := multiPartWriter.FormDataContentType() - err = s.client.putWithMultiPart(endpoint, bodyBuf, contentType) + err = s.pulsar.Client.PutWithMultiPart(endpoint, bodyBuf, contentType) if err != nil { return err } @@ -371,14 +371,14 @@ func (s *sources) UpdateSourceWithURL(config *utils.SourceConfig, pkgURL string, } func (s *sources) DeleteSource(tenant, namespace, source string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - return s.client.delete(endpoint) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source) + return s.pulsar.Client.Delete(endpoint) } func (s *sources) GetSourceStatus(tenant, namespace, source string) (utils.SourceStatus, error) { var sourceStatus utils.SourceStatus - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - err := s.client.get(endpoint+"/status", &sourceStatus) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source) + err := s.pulsar.Client.Get(endpoint+"/status", &sourceStatus) return sourceStatus, err } @@ -386,55 +386,55 @@ func (s *sources) GetSourceStatusWithID(tenant, namespace, source string, id int utils.SourceInstanceStatusData, error) { var sourceInstanceStatusData utils.SourceInstanceStatusData instanceID := fmt.Sprintf("%d", id) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, instanceID) - err := s.client.get(endpoint+"/status", &sourceInstanceStatusData) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source, instanceID) + err := s.pulsar.Client.Get(endpoint+"/status", &sourceInstanceStatusData) return sourceInstanceStatusData, err } func (s *sources) RestartSource(tenant, namespace, source string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - return s.client.post(endpoint+"/restart", "") + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source) + return s.pulsar.Client.Post(endpoint+"/restart", "") } func (s *sources) RestartSourceWithID(tenant, namespace, source string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source, id) - return s.client.post(endpoint+"/restart", "") + return s.pulsar.Client.Post(endpoint+"/restart", "") } func (s *sources) StopSource(tenant, namespace, source string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - return s.client.post(endpoint+"/stop", "") + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source) + return s.pulsar.Client.Post(endpoint+"/stop", "") } func (s *sources) StopSourceWithID(tenant, namespace, source string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source, id) - return s.client.post(endpoint+"/stop", "") + return s.pulsar.Client.Post(endpoint+"/stop", "") } func (s *sources) StartSource(tenant, namespace, source string) error { - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source) - return s.client.post(endpoint+"/start", "") + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source) + return s.pulsar.Client.Post(endpoint+"/start", "") } func (s *sources) StartSourceWithID(tenant, namespace, source string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) - endpoint := s.client.endpoint(s.basePath, tenant, namespace, source, id) + endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source, id) - return s.client.post(endpoint+"/start", "") + return s.pulsar.Client.Post(endpoint+"/start", "") } func (s *sources) GetBuiltInSources() ([]*utils.ConnectorDefinition, error) { var connectorDefinition []*utils.ConnectorDefinition - endpoint := s.client.endpoint(s.basePath, "builtinsources") - err := s.client.get(endpoint, &connectorDefinition) + endpoint := s.pulsar.endpoint(s.basePath, "builtinsources") + err := s.pulsar.Client.Get(endpoint, &connectorDefinition) return connectorDefinition, err } func (s *sources) ReloadBuiltInSources() error { - endpoint := s.client.endpoint(s.basePath, "reloadBuiltInSources") - return s.client.post(endpoint, "") + endpoint := s.pulsar.endpoint(s.basePath, "reloadBuiltInSources") + return s.pulsar.Client.Post(endpoint, "") } diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/pulsar/subscription.go index 5e59686bf1..49a9531f39 100644 --- a/pulsaradmin/pkg/pulsar/subscription.go +++ b/pulsaradmin/pkg/pulsar/subscription.go @@ -71,73 +71,73 @@ type Subscriptions interface { } type subscriptions struct { - client *client + pulsar *pulsarClient basePath string SubPath string } // Subscriptions is used to access the subscriptions endpoints -func (c *client) Subscriptions() Subscriptions { +func (c *pulsarClient) Subscriptions() Subscriptions { return &subscriptions{ - client: c, + pulsar: c, basePath: "", SubPath: "subscription", } } func (s *subscriptions) Create(topic utils.TopicName, sName string, messageID utils.MessageID) error { - endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName)) - return s.client.put(endpoint, messageID) + endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName)) + return s.pulsar.Client.Put(endpoint, messageID) } func (s *subscriptions) Delete(topic utils.TopicName, sName string) error { - endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName)) - return s.client.delete(endpoint) + endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName)) + return s.pulsar.Client.Delete(endpoint) } func (s *subscriptions) List(topic utils.TopicName) ([]string, error) { - endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), "subscriptions") + endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), "subscriptions") var list []string - return list, s.client.get(endpoint, &list) + return list, s.pulsar.Client.Get(endpoint, &list) } func (s *subscriptions) ResetCursorToMessageID(topic utils.TopicName, sName string, id utils.MessageID) error { - endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "resetcursor") - return s.client.post(endpoint, id) + endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "resetcursor") + return s.pulsar.Client.Post(endpoint, id) } func (s *subscriptions) ResetCursorToTimestamp(topic utils.TopicName, sName string, timestamp int64) error { - endpoint := s.client.endpoint( + endpoint := s.pulsar.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "resetcursor", strconv.FormatInt(timestamp, 10)) - return s.client.post(endpoint, "") + return s.pulsar.Client.Post(endpoint, "") } func (s *subscriptions) ClearBacklog(topic utils.TopicName, sName string) error { - endpoint := s.client.endpoint( + endpoint := s.pulsar.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "skip_all") - return s.client.post(endpoint, "") + return s.pulsar.Client.Post(endpoint, "") } func (s *subscriptions) SkipMessages(topic utils.TopicName, sName string, n int64) error { - endpoint := s.client.endpoint( + endpoint := s.pulsar.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "skip", strconv.FormatInt(n, 10)) - return s.client.post(endpoint, "") + return s.pulsar.Client.Post(endpoint, "") } func (s *subscriptions) ExpireMessages(topic utils.TopicName, sName string, expire int64) error { - endpoint := s.client.endpoint( + endpoint := s.pulsar.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "expireMessages", strconv.FormatInt(expire, 10)) - return s.client.post(endpoint, "") + return s.pulsar.Client.Post(endpoint, "") } func (s *subscriptions) ExpireAllMessages(topic utils.TopicName, expire int64) error { - endpoint := s.client.endpoint( + endpoint := s.pulsar.endpoint( s.basePath, topic.GetRestPath(), "all_subscription", "expireMessages", strconv.FormatInt(expire, 10)) - return s.client.post(endpoint, "") + return s.pulsar.Client.Post(endpoint, "") } func (s *subscriptions) PeekMessages(topic utils.TopicName, sName string, n int) ([]*utils.Message, error) { @@ -158,14 +158,10 @@ func (s *subscriptions) PeekMessages(topic utils.TopicName, sName string, n int) } func (s *subscriptions) peekNthMessage(topic utils.TopicName, sName string, pos int) ([]*utils.Message, error) { - endpoint := s.client.endpoint(s.basePath, topic.GetRestPath(), "subscription", url.QueryEscape(sName), + endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), "subscription", url.QueryEscape(sName), "position", strconv.Itoa(pos)) - req, err := s.client.newRequest(http.MethodGet, endpoint) - if err != nil { - return nil, err - } - resp, err := checkSuccessful(s.client.doRequest(req)) + resp, err := s.pulsar.Client.MakeRequest(http.MethodGet, endpoint) if err != nil { return nil, err } @@ -174,6 +170,14 @@ func (s *subscriptions) peekNthMessage(topic utils.TopicName, sName string, pos return handleResp(topic, resp) } +// safeRespClose is used to close a response body +func safeRespClose(resp *http.Response) { + if resp != nil { + // ignore error since it is closing a response body + _ = resp.Body.Close() + } +} + const ( PublishTimeHeader = "X-Pulsar-Publish-Time" BatchHeader = "X-Pulsar-Num-Batch-Message" diff --git a/pulsaradmin/pkg/pulsar/tenant.go b/pulsaradmin/pkg/pulsar/tenant.go index 038a282299..84e133c8cf 100644 --- a/pulsaradmin/pkg/pulsar/tenant.go +++ b/pulsaradmin/pkg/pulsar/tenant.go @@ -40,43 +40,43 @@ type Tenants interface { } type tenants struct { - client *client + pulsar *pulsarClient basePath string } // Tenants is used to access the tenants endpoints -func (c *client) Tenants() Tenants { +func (c *pulsarClient) Tenants() Tenants { return &tenants{ - client: c, + pulsar: c, basePath: "/tenants", } } func (c *tenants) Create(data utils.TenantData) error { - endpoint := c.client.endpoint(c.basePath, data.Name) - return c.client.put(endpoint, &data) + endpoint := c.pulsar.endpoint(c.basePath, data.Name) + return c.pulsar.Client.Put(endpoint, &data) } func (c *tenants) Delete(name string) error { - endpoint := c.client.endpoint(c.basePath, name) - return c.client.delete(endpoint) + endpoint := c.pulsar.endpoint(c.basePath, name) + return c.pulsar.Client.Delete(endpoint) } func (c *tenants) Update(data utils.TenantData) error { - endpoint := c.client.endpoint(c.basePath, data.Name) - return c.client.post(endpoint, &data) + endpoint := c.pulsar.endpoint(c.basePath, data.Name) + return c.pulsar.Client.Post(endpoint, &data) } func (c *tenants) List() ([]string, error) { var tenantList []string - endpoint := c.client.endpoint(c.basePath, "") - err := c.client.get(endpoint, &tenantList) + endpoint := c.pulsar.endpoint(c.basePath, "") + err := c.pulsar.Client.Get(endpoint, &tenantList) return tenantList, err } func (c *tenants) Get(name string) (utils.TenantData, error) { var data utils.TenantData - endpoint := c.client.endpoint(c.basePath, name) - err := c.client.get(endpoint, &data) + endpoint := c.pulsar.endpoint(c.basePath, name) + err := c.pulsar.Client.Get(endpoint, &data) return data, err } diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index e11b1843fe..f5865a3431 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -101,7 +101,7 @@ type Topics interface { } type topics struct { - client *client + pulsar *pulsarClient basePath string persistentPath string nonPersistentPath string @@ -109,9 +109,9 @@ type topics struct { } // Topics is used to access the topics endpoints -func (c *client) Topics() Topics { +func (c *pulsarClient) Topics() Topics { return &topics{ - client: c, + pulsar: c, basePath: "", persistentPath: "/persistent", nonPersistentPath: "/non-persistent", @@ -120,33 +120,33 @@ func (c *client) Topics() Topics { } func (t *topics) Create(topic utils.TopicName, partitions int) error { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "partitions") if partitions == 0 { - endpoint = t.client.endpoint(t.basePath, topic.GetRestPath()) + endpoint = t.pulsar.endpoint(t.basePath, topic.GetRestPath()) } - return t.client.put(endpoint, partitions) + return t.pulsar.Client.Put(endpoint, partitions) } func (t *topics) Delete(topic utils.TopicName, force bool, nonPartitioned bool) error { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "partitions") if nonPartitioned { - endpoint = t.client.endpoint(t.basePath, topic.GetRestPath()) + endpoint = t.pulsar.endpoint(t.basePath, topic.GetRestPath()) } params := map[string]string{ "force": strconv.FormatBool(force), } - return t.client.deleteWithQueryParams(endpoint, nil, params) + return t.pulsar.Client.DeleteWithQueryParams(endpoint, params) } func (t *topics) Update(topic utils.TopicName, partitions int) error { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") - return t.client.post(endpoint, partitions) + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "partitions") + return t.pulsar.Client.Post(endpoint, partitions) } func (t *topics) GetMetadata(topic utils.TopicName) (utils.PartitionedTopicMetadata, error) { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitions") + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "partitions") var partitionedMeta utils.PartitionedTopicMetadata - err := t.client.get(endpoint, &partitionedMeta) + err := t.pulsar.Client.Get(endpoint, &partitionedMeta) return partitionedMeta, err } @@ -156,10 +156,10 @@ func (t *topics) List(namespace utils.NameSpaceName) ([]string, []string, error) nonPartitionedTopicsChan := make(chan []string) errChan := make(chan error) - pp := t.client.endpoint(t.persistentPath, namespace.String(), "partitioned") - np := t.client.endpoint(t.nonPersistentPath, namespace.String(), "partitioned") - p := t.client.endpoint(t.persistentPath, namespace.String()) - n := t.client.endpoint(t.nonPersistentPath, namespace.String()) + pp := t.pulsar.endpoint(t.persistentPath, namespace.String(), "partitioned") + np := t.pulsar.endpoint(t.nonPersistentPath, namespace.String(), "partitioned") + p := t.pulsar.endpoint(t.persistentPath, namespace.String()) + n := t.pulsar.endpoint(t.nonPersistentPath, namespace.String()) go t.getTopics(pp, partitionedTopicsChan, errChan) go t.getTopics(np, partitionedTopicsChan, errChan) @@ -190,114 +190,114 @@ func (t *topics) List(namespace utils.NameSpaceName) ([]string, []string, error) func (t *topics) getTopics(endpoint string, out chan<- []string, err chan<- error) { var topics []string - err <- t.client.get(endpoint, &topics) + err <- t.pulsar.Client.Get(endpoint, &topics) out <- topics } func (t *topics) GetInternalInfo(topic utils.TopicName) (utils.ManagedLedgerInfo, error) { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "internal-info") + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "internal-info") var info utils.ManagedLedgerInfo - err := t.client.get(endpoint, &info) + err := t.pulsar.Client.Get(endpoint, &info) return info, err } func (t *topics) GetPermissions(topic utils.TopicName) (map[string][]common.AuthAction, error) { var permissions map[string][]common.AuthAction - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions") - err := t.client.get(endpoint, &permissions) + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "permissions") + err := t.pulsar.Client.Get(endpoint, &permissions) return permissions, err } func (t *topics) GrantPermission(topic utils.TopicName, role string, action []common.AuthAction) error { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions", role) + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "permissions", role) s := []string{} for _, v := range action { s = append(s, v.String()) } - return t.client.post(endpoint, s) + return t.pulsar.Client.Post(endpoint, s) } func (t *topics) RevokePermission(topic utils.TopicName, role string) error { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "permissions", role) - return t.client.delete(endpoint) + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "permissions", role) + return t.pulsar.Client.Delete(endpoint) } func (t *topics) Lookup(topic utils.TopicName) (utils.LookupData, error) { var lookup utils.LookupData endpoint := fmt.Sprintf("%s/%s", t.lookupPath, topic.GetRestPath()) - err := t.client.get(endpoint, &lookup) + err := t.pulsar.Client.Get(endpoint, &lookup) return lookup, err } func (t *topics) GetBundleRange(topic utils.TopicName) (string, error) { endpoint := fmt.Sprintf("%s/%s/%s", t.lookupPath, topic.GetRestPath(), "bundle") - data, err := t.client.getWithQueryParams(endpoint, nil, nil, false) + data, err := t.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) return string(data), err } func (t *topics) GetLastMessageID(topic utils.TopicName) (utils.MessageID, error) { var messageID utils.MessageID - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "lastMessageId") - err := t.client.get(endpoint, &messageID) + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "lastMessageId") + err := t.pulsar.Client.Get(endpoint, &messageID) return messageID, err } func (t *topics) GetStats(topic utils.TopicName) (utils.TopicStats, error) { var stats utils.TopicStats - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "stats") - err := t.client.get(endpoint, &stats) + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "stats") + err := t.pulsar.Client.Get(endpoint, &stats) return stats, err } func (t *topics) GetInternalStats(topic utils.TopicName) (utils.PersistentTopicInternalStats, error) { var stats utils.PersistentTopicInternalStats - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "internalStats") - err := t.client.get(endpoint, &stats) + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "internalStats") + err := t.pulsar.Client.Get(endpoint, &stats) return stats, err } func (t *topics) GetPartitionedStats(topic utils.TopicName, perPartition bool) (utils.PartitionedTopicStats, error) { var stats utils.PartitionedTopicStats - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "partitioned-stats") + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "partitioned-stats") params := map[string]string{ "perPartition": strconv.FormatBool(perPartition), } - _, err := t.client.getWithQueryParams(endpoint, &stats, params, true) + _, err := t.pulsar.Client.GetWithQueryParams(endpoint, &stats, params, true) return stats, err } func (t *topics) Terminate(topic utils.TopicName) (utils.MessageID, error) { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "terminate") + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "terminate") var messageID utils.MessageID - err := t.client.postWithObj(endpoint, "", &messageID) + err := t.pulsar.Client.PostWithObj(endpoint, "", &messageID) return messageID, err } func (t *topics) Offload(topic utils.TopicName, messageID utils.MessageID) error { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "offload") - return t.client.put(endpoint, messageID) + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "offload") + return t.pulsar.Client.Put(endpoint, messageID) } func (t *topics) OffloadStatus(topic utils.TopicName) (utils.OffloadProcessStatus, error) { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "offload") + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "offload") var status utils.OffloadProcessStatus - err := t.client.get(endpoint, &status) + err := t.pulsar.Client.Get(endpoint, &status) return status, err } func (t *topics) Unload(topic utils.TopicName) error { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "unload") - return t.client.put(endpoint, "") + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "unload") + return t.pulsar.Client.Put(endpoint, "") } func (t *topics) Compact(topic utils.TopicName) error { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "compaction") - return t.client.put(endpoint, "") + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "compaction") + return t.pulsar.Client.Put(endpoint, "") } func (t *topics) CompactStatus(topic utils.TopicName) (utils.LongRunningProcessStatus, error) { - endpoint := t.client.endpoint(t.basePath, topic.GetRestPath(), "compaction") + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "compaction") var status utils.LongRunningProcessStatus - err := t.client.get(endpoint, &status) + err := t.pulsar.Client.Get(endpoint, &status) return status, err } From ef39338866f96a597263bfee31e836209a7dd0f3 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 9 Dec 2019 17:52:29 +0800 Subject: [PATCH 147/348] Support create key pair command (streamnative/pulsarctl#149) --- *Motivation* Support create key pair using RSA and ECDSA signature algorithm. *Modifications* - Add command `create-key-pair` - Add test for the command - Add RSA and ECDSA signature algorithm - Follow the kubectl description style --- pulsaradmin/pkg/pulsar/admin.go | 1 + .../common/algorithm/algorithm/algorithm.go | 62 ++++++++++ .../algorithm/algorithm/algorithm_test.go | 106 ++++++++++++++++++ .../pulsar/common/algorithm/ecdsa/es256.go | 36 ++++++ .../pulsar/common/algorithm/ecdsa/es384.go | 36 ++++++ .../pulsar/common/algorithm/ecdsa/es512.go | 36 ++++++ .../common/algorithm/keypair/keypair.go | 103 +++++++++++++++++ .../pkg/pulsar/common/algorithm/rsa/rs256.go | 35 ++++++ .../pkg/pulsar/common/algorithm/rsa/rs384.go | 35 ++++++ .../pkg/pulsar/common/algorithm/rsa/rs512.go | 35 ++++++ pulsaradmin/pkg/pulsar/token.go | 46 ++++++++ 11 files changed, 531 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es256.go create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs256.go create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs384.go create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs512.go create mode 100644 pulsaradmin/pkg/pulsar/token.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 8b627caf8a..57772a6c7b 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -49,6 +49,7 @@ type Client interface { BrokerStats() BrokerStats ResourceQuotas() ResourceQuotas FunctionsWorker() FunctionsWorker + Token() Token } type pulsarClient struct { diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go new file mode 100644 index 0000000000..818bad0453 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.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 algorithm + +import ( + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/ecdsa" + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/rsa" + + "github.com/pkg/errors" +) + +type Algorithm string + +const ( + RS256 = "RS256" + RS384 = "RS384" + RS512 = "RS512" + ES256 = "ES256" + ES384 = "ES384" + ES512 = "ES512" +) + +var algorithmMap = map[Algorithm]SignatureAlgorithm{ + RS256: new(rsa.RS256), + RS384: new(rsa.RS384), + RS512: new(rsa.RS512), + ES256: new(ecdsa.ES256), + ES384: new(ecdsa.ES384), + ES512: new(ecdsa.ES512), +} + +// SignatureAlgorithm is a collection of all signature algorithm and it provides +// some basic method to use +type SignatureAlgorithm interface { + // GenerateKeyPair generates public and private key + GenerateKeyPair() (*keypair.KeyPair, error) +} + +func GetSignatureAlgorithm(algorithm Algorithm) (SignatureAlgorithm, error) { + sa := algorithmMap[algorithm] + if sa == nil { + return nil, errors.Errorf("the signature algorithm '%s' is invalid. Valid options are: "+ + "'RS256', 'RS384', 'RS512', 'ES256', 'ES384', 'ES512'\n", algorithm) + } + return sa, nil +} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go new file mode 100644 index 0000000000..7b9b225f7b --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go @@ -0,0 +1,106 @@ +// 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 algorithm + +import ( + "fmt" + "testing" + + "github.com/stretchr/testify/assert" +) + +var testData = []struct { + name string + algorithm Algorithm +}{ + {"RSA", RS256}, + {"RSA", RS384}, + {"RSA", RS512}, + {"ECDSA", ES256}, + {"ECDSA", ES384}, + {"ECDSA", ES512}, + {"INVALID", Algorithm("INVALID")}, +} + +func TestGetSignatureAlgorithm(t *testing.T) { + for _, data := range testData { + t.Logf("test case: %+v", data) + switch data.name { + case "RSA": + testRSA(t, data.algorithm) + case "ECDSA": + testECDSA(t, data.algorithm) + default: + sa, err := GetSignatureAlgorithm(data.algorithm) + assert.Nil(t, sa) + assert.NotNil(t, err) + assert.Equal(t, + fmt.Sprintf("the signature algorithm '%s' is invalid. Valid options are: "+ + "'RS256', 'RS384', 'RS512', 'ES256', 'ES384', 'ES512'\n", data.algorithm), + err.Error()) + } + } +} + +func testRSA(t *testing.T, algorithm Algorithm) { + sa, err := GetSignatureAlgorithm(algorithm) + assert.Nil(t, err) + + kp, err := sa.GenerateKeyPair() + assert.Nil(t, err) + assert.NotNil(t, kp) + _, err = kp.EncodedPrivateKey() + assert.Nil(t, err) + _, err = kp.EncodedPublicKey() + assert.Nil(t, err) + + rsaPrivateKey, err := kp.GetRsaPrivateKey() + assert.Nil(t, err) + assert.NotNil(t, rsaPrivateKey) + + ecdsaPrivateKey, err := kp.GetEcdsaPrivateKey() + assert.Nil(t, ecdsaPrivateKey) + assert.NotNil(t, err) + assert.Equal(t, + "the private key is not generated using ECDSA signature algorithm", + err.Error()) +} + +func testECDSA(t *testing.T, algorithm Algorithm) { + sa, err := GetSignatureAlgorithm(algorithm) + assert.Nil(t, err) + + kp, err := sa.GenerateKeyPair() + assert.Nil(t, err) + assert.NotNil(t, kp) + _, err = kp.EncodedPrivateKey() + assert.Nil(t, err) + _, err = kp.EncodedPublicKey() + assert.Nil(t, err) + + ecdsaPrivateKey, err := kp.GetEcdsaPrivateKey() + assert.Nil(t, err) + assert.NotNil(t, ecdsaPrivateKey) + + rsaPrivateKey, err := kp.GetRsaPrivateKey() + assert.Nil(t, rsaPrivateKey) + assert.NotNil(t, err) + assert.Equal(t, + "the private key is not generated using RSA signature algorithm", + err.Error()) +} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es256.go b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es256.go new file mode 100644 index 0000000000..3851ad0144 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es256.go @@ -0,0 +1,36 @@ +// 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 ecdsa + +import ( + "crypto/ecdsa" + "crypto/elliptic" + "crypto/rand" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" +) + +type ES256 struct{} + +func (h *ES256) GenerateKeyPair() (*keypair.KeyPair, error) { + pri, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader) + if err != nil { + return nil, err + } + return keypair.New(keypair.ECDSA, pri), nil +} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go new file mode 100644 index 0000000000..42139b0ab1 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go @@ -0,0 +1,36 @@ +// 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 ecdsa + +import ( + "crypto/ecdsa" + "crypto/elliptic" + "crypto/rand" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" +) + +type ES384 struct{} + +func (h *ES384) GenerateKeyPair() (*keypair.KeyPair, error) { + pri, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader) + if err != nil { + return nil, err + } + return keypair.New(keypair.ECDSA, pri), nil +} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go new file mode 100644 index 0000000000..740127d5b1 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go @@ -0,0 +1,36 @@ +// 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 ecdsa + +import ( + "crypto/ecdsa" + "crypto/elliptic" + "crypto/rand" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" +) + +type ES512 struct{} + +func (h *ES512) GenerateKeyPair() (*keypair.KeyPair, error) { + pri, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader) + if err != nil { + return nil, err + } + return keypair.New(keypair.ECDSA, pri), nil +} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go b/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go new file mode 100644 index 0000000000..f6923aca7f --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go @@ -0,0 +1,103 @@ +// 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 keypair + +import ( + "crypto/ecdsa" + "crypto/rsa" + "crypto/x509" + + "github.com/pkg/errors" +) + +type KeyType int + +const ( + RSA KeyType = iota + ECDSA +) + +// KeyPair saves the ecdsa private key or the rsa private key and provides +// a get public/private encoded bytes array method +type KeyPair struct { + keyType KeyType + privateKey interface{} +} + +func New(keyType KeyType, privateKey interface{}) *KeyPair { + return &KeyPair{ + keyType: keyType, + privateKey: privateKey, + } +} + +// EncodedPrivateKey gets the encoded private key +func (k *KeyPair) EncodedPrivateKey() ([]byte, error) { + switch k.keyType { + case RSA: + key, err := k.GetRsaPrivateKey() + if err != nil { + return nil, err + } + return x509.MarshalPKCS1PrivateKey(key), err + case ECDSA: + key, err := k.GetEcdsaPrivateKey() + if err != nil { + return nil, err + } + return x509.MarshalECPrivateKey(key) + } + return nil, errors.New("unknown error") +} + +// EncodedPublicKey gets the encoded public key +func (k *KeyPair) EncodedPublicKey() ([]byte, error) { + switch k.keyType { + case RSA: + key, err := k.GetRsaPrivateKey() + return x509.MarshalPKCS1PublicKey(&key.PublicKey), err + case ECDSA: + key, _ := k.GetEcdsaPrivateKey() + return x509.MarshalPKIXPublicKey(&key.PublicKey) + } + return nil, errors.New("unknown error") +} + +// GetRsaPrivateKey gets the rsa private key if you are using rsa signature +// algorithm to generate the private key +func (k *KeyPair) GetRsaPrivateKey() (*rsa.PrivateKey, error) { + if k.keyType != RSA { + return nil, errors.New("the private key is not generated using RSA signature algorithm") + } + if rsaKey, ok := k.privateKey.(*rsa.PrivateKey); ok { + return rsaKey, nil + } + return nil, errors.New("the private key is not generated using RSA signature algorithm") +} + +// GetEcdsaPrivateKey gets the ecdsa private key if you are using ecdsa signature +// algorithm to generate the private key +func (k *KeyPair) GetEcdsaPrivateKey() (*ecdsa.PrivateKey, error) { + if k.keyType != ECDSA { + return nil, errors.New("the private key is not generated using ECDSA signature algorithm") + } + if ecdsaKey, ok := k.privateKey.(*ecdsa.PrivateKey); ok { + return ecdsaKey, nil + } + return nil, errors.New("the private key is not generated using ecdsa signature algorithm") +} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs256.go b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs256.go new file mode 100644 index 0000000000..a5ccd6f52a --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs256.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 rsa + +import ( + "crypto/rand" + "crypto/rsa" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" +) + +type RS256 struct{} + +func (p *RS256) GenerateKeyPair() (*keypair.KeyPair, error) { + pri, err := rsa.GenerateKey(rand.Reader, 2048) + if err != nil { + return nil, err + } + return keypair.New(keypair.RSA, pri), nil +} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs384.go b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs384.go new file mode 100644 index 0000000000..d1794664cc --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs384.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 rsa + +import ( + "crypto/rand" + "crypto/rsa" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" +) + +type RS384 struct{} + +func (p *RS384) GenerateKeyPair() (*keypair.KeyPair, error) { + pri, err := rsa.GenerateKey(rand.Reader, 3072) + if err != nil { + return nil, err + } + return keypair.New(keypair.RSA, pri), nil +} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs512.go b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs512.go new file mode 100644 index 0000000000..891861246b --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs512.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 rsa + +import ( + "crypto/rand" + "crypto/rsa" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" +) + +type RS512 struct{} + +func (p *RS512) GenerateKeyPair() (*keypair.KeyPair, error) { + pri, err := rsa.GenerateKey(rand.Reader, 4096) + if err != nil { + return nil, err + } + return keypair.New(keypair.RSA, pri), nil +} diff --git a/pulsaradmin/pkg/pulsar/token.go b/pulsaradmin/pkg/pulsar/token.go new file mode 100644 index 0000000000..8991eee2a3 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/token.go @@ -0,0 +1,46 @@ +// 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 pulsar + +import ( + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/algorithm" + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" +) + +type Token interface { + // CreateKeyPair is used to create public and private key pair using the given signature algorithm + CreateKeyPair(algorithm.Algorithm) (*keypair.KeyPair, error) +} + +type token struct { + pulsar *pulsarClient +} + +func (c *pulsarClient) Token() Token { + return &token{ + pulsar: c, + } +} + +func (c *token) CreateKeyPair(signatureAlgorithm algorithm.Algorithm) (*keypair.KeyPair, error) { + sa, err := algorithm.GetSignatureAlgorithm(signatureAlgorithm) + if err != nil { + return nil, err + } + return sa.GenerateKeyPair() +} From ebf02a54c98768c8e5c684ffb0e6c3c6015f2d71 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 11 Dec 2019 01:08:22 +0800 Subject: [PATCH 148/348] Support create secret key command (streamnative/pulsarctl#145) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit *Motivation* Support create secret key command. Now can use `pulsarctl token create-secret-key` to generate a secret key. *Modifications* - Add command `create-secret-key` - Add test for the command *output* ``` USED FOR: This command is used for creating a secret key. REQUIRED PERMISSION: This command does not need any permission. OUTPUT: #normal output [0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0] #Write the secret key to the terminal and encode with base64 AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA= #Write the secret key to a file Write secret to file (filename) successfully. #Using invalid signature algorithm to generate secret key [✖] the signature algorithm '(signature algorithm)' is invalid. Valid options include: 'HS256', 'HS384', 'HS512' #Not using HMAC signature algorithm to generate secret key [✖] the signature algorithm '(signature algorithm)' is invalid. Valid options include: 'HS256', 'HS384', 'HS512' Usage: pulsarctl token create-secret-key [flags] Examples: #Create a secret key and print it to the terminal pulsarctl token create-secret-key #Create a secret key and print it with base64 encode to the terminal pulsarctl token create-secret-key --base64 #Create a secret key and output to a file pulsarctl token create-secret-key --output (file path) Create secret key flags: -a, --signature-algorithm string The signature algorithm of generate secret key, valid options include 'HS256', 'HS384', 'HS512' (default "HS256") -o, --output-file string The file that the secret key write to -b, --base64 Using base64 to encode the secret key ``` --- .../common/algorithm/algorithm/algorithm.go | 26 ++++++++---- .../algorithm/algorithm/algorithm_test.go | 39 +++++++++++++++++- .../pulsar/common/algorithm/ecdsa/es256.go | 6 +++ .../pulsar/common/algorithm/ecdsa/es384.go | 6 +++ .../pulsar/common/algorithm/ecdsa/es512.go | 6 +++ .../pkg/pulsar/common/algorithm/hmac/hs256.go | 41 +++++++++++++++++++ .../pkg/pulsar/common/algorithm/hmac/hs384.go | 41 +++++++++++++++++++ .../pkg/pulsar/common/algorithm/hmac/hs512.go | 41 +++++++++++++++++++ .../pkg/pulsar/common/algorithm/rsa/rs256.go | 6 +++ .../pkg/pulsar/common/algorithm/rsa/rs384.go | 6 +++ .../pkg/pulsar/common/algorithm/rsa/rs512.go | 6 +++ pulsaradmin/pkg/pulsar/token.go | 11 +++++ 12 files changed, 225 insertions(+), 10 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs256.go create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs384.go create mode 100644 pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs512.go diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go index 818bad0453..6cb5bd33fc 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go @@ -19,6 +19,7 @@ package algorithm import ( "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/ecdsa" + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/hmac" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/rsa" @@ -28,15 +29,21 @@ import ( type Algorithm string const ( - RS256 = "RS256" - RS384 = "RS384" - RS512 = "RS512" - ES256 = "ES256" - ES384 = "ES384" - ES512 = "ES512" + HS256 Algorithm = "HS256" + HS384 Algorithm = "HS384" + HS512 Algorithm = "HS512" + RS256 Algorithm = "RS256" + RS384 Algorithm = "RS384" + RS512 Algorithm = "RS512" + ES256 Algorithm = "ES256" + ES384 Algorithm = "ES384" + ES512 Algorithm = "ES512" ) var algorithmMap = map[Algorithm]SignatureAlgorithm{ + HS256: new(hmac.HS256), + HS384: new(hmac.HS384), + HS512: new(hmac.HS512), RS256: new(rsa.RS256), RS384: new(rsa.RS384), RS512: new(rsa.RS512), @@ -50,13 +57,16 @@ var algorithmMap = map[Algorithm]SignatureAlgorithm{ type SignatureAlgorithm interface { // GenerateKeyPair generates public and private key GenerateKeyPair() (*keypair.KeyPair, error) + + // GenerateSecret is used to generating a secret + GenerateSecret() ([]byte, error) } func GetSignatureAlgorithm(algorithm Algorithm) (SignatureAlgorithm, error) { sa := algorithmMap[algorithm] if sa == nil { - return nil, errors.Errorf("the signature algorithm '%s' is invalid. Valid options are: "+ - "'RS256', 'RS384', 'RS512', 'ES256', 'ES384', 'ES512'\n", algorithm) + return nil, errors.Errorf("the signature algorithm '%s' is invalid. Valid options are: 'HS256', "+ + "'HS384', 'HS512', 'RS256', 'RS384', 'RS512', 'ES256', 'ES384', 'ES512'\n", algorithm) } return sa, nil } diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go index 7b9b225f7b..50a77423fa 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go @@ -28,6 +28,9 @@ var testData = []struct { name string algorithm Algorithm }{ + {"HMAC", HS256}, + {"HMAC", HS384}, + {"HMAC", HS512}, {"RSA", RS256}, {"RSA", RS384}, {"RSA", RS512}, @@ -41,6 +44,8 @@ func TestGetSignatureAlgorithm(t *testing.T) { for _, data := range testData { t.Logf("test case: %+v", data) switch data.name { + case "HMAC": + testHMAC(t, data.algorithm) case "RSA": testRSA(t, data.algorithm) case "ECDSA": @@ -50,13 +55,33 @@ func TestGetSignatureAlgorithm(t *testing.T) { assert.Nil(t, sa) assert.NotNil(t, err) assert.Equal(t, - fmt.Sprintf("the signature algorithm '%s' is invalid. Valid options are: "+ - "'RS256', 'RS384', 'RS512', 'ES256', 'ES384', 'ES512'\n", data.algorithm), + fmt.Sprintf("the signature algorithm '%s' is invalid. Valid options are: 'HS256', "+ + "'HS384', 'HS512', 'RS256', 'RS384', 'RS512', 'ES256', 'ES384', 'ES512'\n", data.algorithm), err.Error()) } } } +func testHMAC(t *testing.T, algorithm Algorithm) { + sa, err := GetSignatureAlgorithm(algorithm) + assert.Nil(t, err) + secrets, err := sa.GenerateSecret() + assert.Nil(t, err) + switch algorithm { + case HS256: + assert.Equal(t, 32, len(secrets)) + case HS384: + assert.Equal(t, 48, len(secrets)) + case HS512: + assert.Equal(t, 64, len(secrets)) + } + + kp, err := sa.GenerateKeyPair() + assert.Nil(t, kp) + assert.NotNil(t, err) + assert.Equal(t, "unsupported operation", err.Error()) +} + func testRSA(t *testing.T, algorithm Algorithm) { sa, err := GetSignatureAlgorithm(algorithm) assert.Nil(t, err) @@ -79,6 +104,11 @@ func testRSA(t *testing.T, algorithm Algorithm) { assert.Equal(t, "the private key is not generated using ECDSA signature algorithm", err.Error()) + + secrets, err := sa.GenerateSecret() + assert.Nil(t, secrets) + assert.NotNil(t, err) + assert.Equal(t, "unsupported operation", err.Error()) } func testECDSA(t *testing.T, algorithm Algorithm) { @@ -103,4 +133,9 @@ func testECDSA(t *testing.T, algorithm Algorithm) { assert.Equal(t, "the private key is not generated using RSA signature algorithm", err.Error()) + + secrets, err := sa.GenerateSecret() + assert.Nil(t, secrets) + assert.NotNil(t, err) + assert.Equal(t, "unsupported operation", err.Error()) } diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es256.go b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es256.go index 3851ad0144..0e652e6e01 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es256.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es256.go @@ -23,10 +23,16 @@ import ( "crypto/rand" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + + "github.com/pkg/errors" ) type ES256 struct{} +func (h *ES256) GenerateSecret() ([]byte, error) { + return nil, errors.New("unsupported operation") +} + func (h *ES256) GenerateKeyPair() (*keypair.KeyPair, error) { pri, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader) if err != nil { diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go index 42139b0ab1..e545f6e8b9 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go @@ -23,10 +23,16 @@ import ( "crypto/rand" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + + "github.com/pkg/errors" ) type ES384 struct{} +func (h *ES384) GenerateSecret() ([]byte, error) { + return nil, errors.New("unsupported operation") +} + func (h *ES384) GenerateKeyPair() (*keypair.KeyPair, error) { pri, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader) if err != nil { diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go index 740127d5b1..5e3211d4e1 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go @@ -23,10 +23,16 @@ import ( "crypto/rand" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + + "github.com/pkg/errors" ) type ES512 struct{} +func (h *ES512) GenerateSecret() ([]byte, error) { + return nil, errors.New("unsupported operation") +} + func (h *ES512) GenerateKeyPair() (*keypair.KeyPair, error) { pri, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader) if err != nil { diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs256.go b/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs256.go new file mode 100644 index 0000000000..b2990cca67 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs256.go @@ -0,0 +1,41 @@ +// 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 hmac + +import ( + "crypto/hmac" + "crypto/rand" + "crypto/sha256" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + + "github.com/pkg/errors" +) + +type HS256 struct{} + +func (h *HS256) GenerateSecret() ([]byte, error) { + bytes := make([]byte, 32) + rand.Read(bytes) + s := hmac.New(sha256.New, bytes) + return s.Sum(nil), nil +} + +func (h *HS256) GenerateKeyPair() (*keypair.KeyPair, error) { + return nil, errors.New("unsupported operation") +} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs384.go b/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs384.go new file mode 100644 index 0000000000..ad226be962 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs384.go @@ -0,0 +1,41 @@ +// 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 hmac + +import ( + "crypto/hmac" + "crypto/rand" + "crypto/sha512" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + + "github.com/pkg/errors" +) + +type HS384 struct{} + +func (h *HS384) GenerateSecret() ([]byte, error) { + bytes := make([]byte, 48) + rand.Read(bytes) + s := hmac.New(sha512.New384, bytes) + return s.Sum(nil), nil +} + +func (h *HS384) GenerateKeyPair() (*keypair.KeyPair, error) { + return nil, errors.New("unsupported operation") +} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs512.go b/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs512.go new file mode 100644 index 0000000000..ed61310519 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs512.go @@ -0,0 +1,41 @@ +// 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 hmac + +import ( + "crypto/hmac" + "crypto/rand" + "crypto/sha512" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + + "github.com/pkg/errors" +) + +type HS512 struct{} + +func (h *HS512) GenerateSecret() ([]byte, error) { + bytes := make([]byte, 64) + rand.Read(bytes) + s := hmac.New(sha512.New, bytes) + return s.Sum(nil), nil +} + +func (h *HS512) GenerateKeyPair() (*keypair.KeyPair, error) { + return nil, errors.New("unsupported operation") +} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs256.go b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs256.go index a5ccd6f52a..e104039e38 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs256.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs256.go @@ -22,10 +22,16 @@ import ( "crypto/rsa" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + + "github.com/pkg/errors" ) type RS256 struct{} +func (p *RS256) GenerateSecret() ([]byte, error) { + return nil, errors.New("unsupported operation") +} + func (p *RS256) GenerateKeyPair() (*keypair.KeyPair, error) { pri, err := rsa.GenerateKey(rand.Reader, 2048) if err != nil { diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs384.go b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs384.go index d1794664cc..a4a8280308 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs384.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs384.go @@ -22,10 +22,16 @@ import ( "crypto/rsa" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + + "github.com/pkg/errors" ) type RS384 struct{} +func (p *RS384) GenerateSecret() ([]byte, error) { + return nil, errors.New("unsupported operation") +} + func (p *RS384) GenerateKeyPair() (*keypair.KeyPair, error) { pri, err := rsa.GenerateKey(rand.Reader, 3072) if err != nil { diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs512.go b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs512.go index 891861246b..5aa5450c74 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs512.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs512.go @@ -22,10 +22,16 @@ import ( "crypto/rsa" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + + "github.com/pkg/errors" ) type RS512 struct{} +func (p *RS512) GenerateSecret() ([]byte, error) { + return nil, errors.New("unsupported operation") +} + func (p *RS512) GenerateKeyPair() (*keypair.KeyPair, error) { pri, err := rsa.GenerateKey(rand.Reader, 4096) if err != nil { diff --git a/pulsaradmin/pkg/pulsar/token.go b/pulsaradmin/pkg/pulsar/token.go index 8991eee2a3..9d8c1dc0de 100644 --- a/pulsaradmin/pkg/pulsar/token.go +++ b/pulsaradmin/pkg/pulsar/token.go @@ -25,6 +25,9 @@ import ( type Token interface { // CreateKeyPair is used to create public and private key pair using the given signature algorithm CreateKeyPair(algorithm.Algorithm) (*keypair.KeyPair, error) + + // CreateSecretKey is used for creating a secret key + CreateSecretKey(algorithm.Algorithm) ([]byte, error) } type token struct { @@ -44,3 +47,11 @@ func (c *token) CreateKeyPair(signatureAlgorithm algorithm.Algorithm) (*keypair. } return sa.GenerateKeyPair() } + +func (c *token) CreateSecretKey(signatureAlgorithm algorithm.Algorithm) ([]byte, error) { + sa, err := algorithm.GetSignatureAlgorithm(signatureAlgorithm) + if err != nil { + return nil, err + } + return sa.GenerateSecret() +} From 58efddb3bf444e3b64ed0c4c11fd0218d8ab61bb Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 11 Dec 2019 15:50:23 +0800 Subject: [PATCH 149/348] Support create token command (streamnative/pulsarctl#150) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit *Motivation* Support create a token using secret key and private key. *Modifications* - Add command `create` - Add test for the command `create` *Command output* ``` ➜ pulsarctl-yong git:(create-token) ./pulsarctl token create -h USED FOR: This command is used for create a token string. REQUIRED PERMISSION: This command does not need any permission. OUTPUT: #Create a token successfully. eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJoZWxsby10ZXN0In0.qxaczygeZaZDlK7jQHHXCaQRbwd2wxIHjCH3y_Lo2Q4 #None of the signing keys is specified. [✖] none of the signing keys is specified #Signing key is specified more than one. [✖] the signing key is specified more than one Usage: pulsarctl token create [flags] Examples: #Create a token using a secret key string. pulsarctl token create --secret-key-string (secret-key-string) --subject (subject) #Create a token using a secret key file. pulsarctl token create --secret-key-file (secret-key-file-path) --subject (subject) #Create a token using a private key file. pulsarctl token create --private-key-file (private-key-file-path) --subject (subject) #Create a token with expire time. pulsarctl token create --secret-key-string (secret-key-string) --subject (subject) --expire 1m #Create a token using a base64 encoded secret key. pulsarctl token create --secret-key-string (secret-key-string) --base64 --subject (subject) Create a token flags: -a, --signature-algorithm string The signature algorithm used to generate the secret key or the private key, generate a token needs to use the same signature algorithm. Valid options are: 'HS256', 'HS384', 'HS512', 'RS256', 'RS384', 'RS512', 'PS256', 'PS384', 'PS512', 'ES256', 'ES384', 'ES512'. (default "RS256") --secret-key-string string The secret key string that used to sign a token. --secret-key-file string The secret key file that used to sign a token. --private-key-file string The private key file that used to sign a toke. --subject string The 'subject' or 'principal' associate with this token. --expire string The expire time for a token. e.g. 1s, 1m, 1h --base64 The secret key is base64 encoded or not. ``` --- pulsaradmin/go.mod | 1 + .../common/algorithm/algorithm/algorithm.go | 3 ++ .../pulsar/common/algorithm/ecdsa/es384.go | 2 +- .../pulsar/common/algorithm/ecdsa/es512.go | 2 +- .../common/algorithm/keypair/keypair.go | 21 ++++++++ pulsaradmin/pkg/pulsar/token.go | 54 ++++++++++++++++++- 6 files changed, 79 insertions(+), 4 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index 601b0a9168..de596d5950 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -4,6 +4,7 @@ go 1.12 require ( github.com/davecgh/go-spew v1.1.1 + github.com/dgrijalva/jwt-go v3.2.0+incompatible github.com/fatih/color v1.7.0 // indirect github.com/golang/protobuf v1.3.1 github.com/google/go-cmp v0.3.1 // indirect diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go index 6cb5bd33fc..75ff5dac19 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go @@ -38,6 +38,9 @@ const ( ES256 Algorithm = "ES256" ES384 Algorithm = "ES384" ES512 Algorithm = "ES512" + PS256 Algorithm = "PS256" + PS384 Algorithm = "PS384" + PS512 Algorithm = "PS512" ) var algorithmMap = map[Algorithm]SignatureAlgorithm{ diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go index e545f6e8b9..b2fdd20444 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go @@ -34,7 +34,7 @@ func (h *ES384) GenerateSecret() ([]byte, error) { } func (h *ES384) GenerateKeyPair() (*keypair.KeyPair, error) { - pri, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader) + pri, err := ecdsa.GenerateKey(elliptic.P384(), rand.Reader) if err != nil { return nil, err } diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go index 5e3211d4e1..bc50349f06 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go @@ -34,7 +34,7 @@ func (h *ES512) GenerateSecret() ([]byte, error) { } func (h *ES512) GenerateKeyPair() (*keypair.KeyPair, error) { - pri, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader) + pri, err := ecdsa.GenerateKey(elliptic.P521(), rand.Reader) if err != nil { return nil, err } diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go b/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go index f6923aca7f..353510db1e 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go @@ -65,6 +65,27 @@ func (k *KeyPair) EncodedPrivateKey() ([]byte, error) { return nil, errors.New("unknown error") } +// DecodePrivateKey parses the private key to a KeyPair +func DecodePrivateKey(keyType KeyType, privateKey []byte) (*KeyPair, error) { + switch keyType { + case RSA: + key, err := x509.ParsePKCS1PrivateKey(privateKey) + if err != nil { + k, e := x509.ParsePKCS8PrivateKey(privateKey) + return New(keyType, k), e + } + return New(keyType, key), nil + case ECDSA: + key, err := x509.ParseECPrivateKey(privateKey) + if err != nil { + k, e := x509.ParsePKCS8PrivateKey(privateKey) + return New(keyType, k), e + } + return New(ECDSA, key), nil + } + return nil, errors.New("unknown error") +} + // EncodedPublicKey gets the encoded public key func (k *KeyPair) EncodedPublicKey() ([]byte, error) { switch k.keyType { diff --git a/pulsaradmin/pkg/pulsar/token.go b/pulsaradmin/pkg/pulsar/token.go index 9d8c1dc0de..7a3e9c67a5 100644 --- a/pulsaradmin/pkg/pulsar/token.go +++ b/pulsaradmin/pkg/pulsar/token.go @@ -20,6 +20,8 @@ package pulsar import ( "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/algorithm" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + + "github.com/dgrijalva/jwt-go" ) type Token interface { @@ -28,6 +30,10 @@ type Token interface { // CreateSecretKey is used for creating a secret key CreateSecretKey(algorithm.Algorithm) ([]byte, error) + + // Create creates a token object using the specified signature algorithm, private key, + // object and the expire time + Create(algorithm.Algorithm, interface{}, string, int64) (string, error) } type token struct { @@ -40,7 +46,7 @@ func (c *pulsarClient) Token() Token { } } -func (c *token) CreateKeyPair(signatureAlgorithm algorithm.Algorithm) (*keypair.KeyPair, error) { +func (t *token) CreateKeyPair(signatureAlgorithm algorithm.Algorithm) (*keypair.KeyPair, error) { sa, err := algorithm.GetSignatureAlgorithm(signatureAlgorithm) if err != nil { return nil, err @@ -48,10 +54,54 @@ func (c *token) CreateKeyPair(signatureAlgorithm algorithm.Algorithm) (*keypair. return sa.GenerateKeyPair() } -func (c *token) CreateSecretKey(signatureAlgorithm algorithm.Algorithm) ([]byte, error) { +func (t *token) CreateSecretKey(signatureAlgorithm algorithm.Algorithm) ([]byte, error) { sa, err := algorithm.GetSignatureAlgorithm(signatureAlgorithm) if err != nil { return nil, err } return sa.GenerateSecret() } + +func (t *token) Create(algorithm algorithm.Algorithm, signKey interface{}, subject string, + expireTime int64) (string, error) { + + claims := &jwt.StandardClaims{ + Subject: subject, + ExpiresAt: expireTime, + } + signMethod := parseAlgorithmToJwtSignMethod(algorithm) + tokenString := jwt.NewWithClaims(signMethod, claims) + + return tokenString.SignedString(signKey) +} + +func parseAlgorithmToJwtSignMethod(a algorithm.Algorithm) jwt.SigningMethod { + switch a { + case algorithm.HS256: + return jwt.SigningMethodHS256 + case algorithm.HS384: + return jwt.SigningMethodHS384 + case algorithm.HS512: + return jwt.SigningMethodHS512 + case algorithm.RS256: + return jwt.SigningMethodRS256 + case algorithm.RS384: + return jwt.SigningMethodRS384 + case algorithm.RS512: + return jwt.SigningMethodRS512 + case algorithm.ES256: + return jwt.SigningMethodES256 + case algorithm.ES384: + return jwt.SigningMethodES384 + case algorithm.ES512: + return jwt.SigningMethodES512 + case algorithm.PS256: + return jwt.SigningMethodPS256 + case algorithm.PS384: + return jwt.SigningMethodPS384 + case algorithm.PS512: + return jwt.SigningMethodPS512 + default: + return jwt.SigningMethodRS256 + } +} From 38d126c77deb83281a00529e18c94b0135212348 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 12 Dec 2019 04:16:02 +0800 Subject: [PATCH 150/348] Support validate token command (streamnative/pulsarctl#151) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit *Motivation* Support validate token command. *Modifications* - Add command `validate` - Add test for the command *Command output* ``` USED FOR: This command is used for validating a token. REQUIRED PERMISSION: This command does not need any permission. OUTPUT: #The token is valid. The subject is (subject), and the expire time is (time). #Both the token string and the token file are not specified. [✖] both the token string and the token file are not specified #Both the token string and the token file are specified. [✖] both the token string and token file are specified #There is no key to validate the token. [✖] none of the validate keys is specified #The key used to validate token is specified more than one. [✖] the validate key is specified more than one Usage: pulsarctl token validate [flags] Examples: #Validate a token string using the specified secret key string. pulsarctl token validate --token-string (token) --secret-key-string (secret-key-string) #Validate a token file using the specified secret key file. pulsarctl token validate --token-string (token) --secret-key-file (secret-key-file-path) #Validate a token string using the specified public key file. pulsarctl token validate --token-string (token) --public-key-file (public-key-file-path) #Validate a token string using the specified base64 encoded secret key string. pulsarctl token validate --token-string (token) --secret-key-string (secret-key-string) --base64 #Validate a token file that signed with the specified secret key string and the specified signature algorithm. pulsarctl toke validate --token-string (token) --secret-key-file (secret-key-file-path) --signature-algorithm (algorithm) Validate a token flags: -a, --signature-algorithm string The signature algorithm is used for generating the token. Valid options are: 'HS256', 'HS384', 'HS512', 'RS256', 'RS384', 'RS512', 'PS256', 'PS384', 'PS512', 'ES256', 'ES384', 'ES512'. (default "RS256") --token-string string The token string that will be validated. --token-file string The token file that will be validated. --secret-key-string string The secret key string that used to validate a token. --secret-key-file string The secret key file that used to validate a token. --public-key-file string The public key file that used to validate a token. --base64 The secret key is base64 encoded or not. ``` --- .../common/algorithm/keypair/keypair.go | 45 ++++++++++++------- pulsaradmin/pkg/pulsar/token.go | 25 +++++++++++ 2 files changed, 53 insertions(+), 17 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go b/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go index 353510db1e..af25265a01 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go @@ -48,24 +48,10 @@ func New(keyType KeyType, privateKey interface{}) *KeyPair { // EncodedPrivateKey gets the encoded private key func (k *KeyPair) EncodedPrivateKey() ([]byte, error) { - switch k.keyType { - case RSA: - key, err := k.GetRsaPrivateKey() - if err != nil { - return nil, err - } - return x509.MarshalPKCS1PrivateKey(key), err - case ECDSA: - key, err := k.GetEcdsaPrivateKey() - if err != nil { - return nil, err - } - return x509.MarshalECPrivateKey(key) - } - return nil, errors.New("unknown error") + return x509.MarshalPKCS8PrivateKey(k.privateKey) } -// DecodePrivateKey parses the private key to a KeyPair +// DecodePrivateKey parses the private key to a KeyPair. func DecodePrivateKey(keyType KeyType, privateKey []byte) (*KeyPair, error) { switch keyType { case RSA: @@ -91,7 +77,10 @@ func (k *KeyPair) EncodedPublicKey() ([]byte, error) { switch k.keyType { case RSA: key, err := k.GetRsaPrivateKey() - return x509.MarshalPKCS1PublicKey(&key.PublicKey), err + if err != nil { + return nil, err + } + return x509.MarshalPKIXPublicKey(&key.PublicKey) case ECDSA: key, _ := k.GetEcdsaPrivateKey() return x509.MarshalPKIXPublicKey(&key.PublicKey) @@ -99,6 +88,28 @@ func (k *KeyPair) EncodedPublicKey() ([]byte, error) { return nil, errors.New("unknown error") } +// DecodeRSAPublicKey parses the rsa public key. +func DecodeRSAPublicKey(publicKey []byte) (*rsa.PublicKey, error) { + pri, err := x509.ParsePKIXPublicKey(publicKey) + if err != nil { + p, e := x509.ParsePKCS1PublicKey(publicKey) + return p, e + } + return pri.(*rsa.PublicKey), err +} + +// DecodeECDSAPublicKey parses the ecdsa public key +func DecodeECDSAPublicKey(publicKey []byte) (*ecdsa.PublicKey, error) { + pubKey, err := x509.ParsePKIXPublicKey(publicKey) + if err != nil { + return nil, err + } + if ecdsaPubKey, ok := pubKey.(*ecdsa.PublicKey); ok { + return ecdsaPubKey, nil + } + return nil, errors.New("the public key is not generated using ECDSA signature algorithm") +} + // GetRsaPrivateKey gets the rsa private key if you are using rsa signature // algorithm to generate the private key func (k *KeyPair) GetRsaPrivateKey() (*rsa.PrivateKey, error) { diff --git a/pulsaradmin/pkg/pulsar/token.go b/pulsaradmin/pkg/pulsar/token.go index 7a3e9c67a5..3bcdb764fe 100644 --- a/pulsaradmin/pkg/pulsar/token.go +++ b/pulsaradmin/pkg/pulsar/token.go @@ -22,6 +22,7 @@ import ( "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" "github.com/dgrijalva/jwt-go" + "github.com/pkg/errors" ) type Token interface { @@ -34,6 +35,9 @@ type Token interface { // Create creates a token object using the specified signature algorithm, private key, // object and the expire time Create(algorithm.Algorithm, interface{}, string, int64) (string, error) + + // Validate a token is valid or not + Validate(algorithm.Algorithm, string, interface{}) (string, int64, error) } type token struct { @@ -75,6 +79,27 @@ func (t *token) Create(algorithm algorithm.Algorithm, signKey interface{}, subje return tokenString.SignedString(signKey) } +func (t *token) Validate(algorithm algorithm.Algorithm, tokenString string, + signKey interface{}) (string, int64, error) { + + // verify the signature algorithm + parsedToken, err := jwt.ParseWithClaims(tokenString, &jwt.StandardClaims{}, + func(jt *jwt.Token) (i interface{}, e error) { + signMethod := parseAlgorithmToJwtSignMethod(algorithm) + if jt.Method != signMethod { + return nil, errors.Errorf("unexpected signing method: %s", algorithm) + } + return signKey, nil + }) + + // get the subject and the expire time + if claim, ok := parsedToken.Claims.(*jwt.StandardClaims); parsedToken.Valid && ok { + return claim.Subject, claim.ExpiresAt, nil + } + + return "", 0, err +} + func parseAlgorithmToJwtSignMethod(a algorithm.Algorithm) jwt.SigningMethod { switch a { case algorithm.HS256: From 07657c34af9844a4777dea7a47ec651b3ddbaf5b Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 16 Dec 2019 02:15:10 +0800 Subject: [PATCH 151/348] Support show a token content (streamnative/pulsarctl#152) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit *Motivation* Support command `show` to show a token algorithm and subject. *Modifications* - Add command `show` - Add test for the command *Command output* ``` USED FOR: This command is used for showing the content of a token. REQUIRED PERMISSION: This command does not need any permission. OUTPUT: #Show the content of the given token. The algorithm and subject of the token are (signature algorithm), (subject). #There is no token to show. [✖] both the token string and the token file are not specified #Too many tokens to show. [✖] both the token string and token file are specified Usage: pulsarctl token show [flags] Examples: #Read a token from env TOKEN. pulsarctl token show #Read a token from the given string. pulsarctl token show --token-string (token) #Read a token from a given file. pulsarctl token show --token-file (token) Show token flags: --token-string string The token string you would like to show the content. --token-file string The token file you would like to show the content. ``` --- pulsaradmin/pkg/pulsar/token.go | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/token.go b/pulsaradmin/pkg/pulsar/token.go index 3bcdb764fe..0ac946185d 100644 --- a/pulsaradmin/pkg/pulsar/token.go +++ b/pulsaradmin/pkg/pulsar/token.go @@ -18,6 +18,8 @@ package pulsar import ( + "strings" + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/algorithm" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" @@ -38,6 +40,12 @@ type Token interface { // Validate a token is valid or not Validate(algorithm.Algorithm, string, interface{}) (string, int64, error) + + // GetAlgorithm gets which algorithm the token used + GetAlgorithm(string) (string, error) + + // GetSubject gets the subject of a token + GetSubject(string) (string, error) } type token struct { @@ -100,6 +108,24 @@ func (t *token) Validate(algorithm algorithm.Algorithm, tokenString string, return "", 0, err } +func (t *token) GetAlgorithm(tokenString string) (string, error) { + parts := strings.Split(tokenString, ".") + algorithm, err := jwt.DecodeSegment(parts[0]) + if err != nil { + return "", err + } + return string(algorithm), nil +} + +func (t *token) GetSubject(tokenString string) (string, error) { + parts := strings.Split(tokenString, ".") + algorithm, err := jwt.DecodeSegment(parts[1]) + if err != nil { + return "", err + } + return string(algorithm), nil +} + func parseAlgorithmToJwtSignMethod(a algorithm.Algorithm) jwt.SigningMethod { switch a { case algorithm.HS256: From aa0d4f852e97762fc3a69d344ff7b39dd566df9a Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 20 Dec 2019 10:01:38 +0800 Subject: [PATCH 152/348] Fixes error handle for bookKeeper error message (streamnative/pulsarctl#155) *Motivation* The BookKeeper error message in the response body and can not parse by the `error` struct. So we need to support to get the error message from the response body and create a new error with the error messsage. *Modifications* - Support read error message from response body --- pulsaradmin/pkg/cli/client.go | 7 +++-- pulsaradmin/pkg/cli/errors.go | 2 +- pulsaradmin/pkg/pulsar/common/errors.go | 36 ------------------------- 3 files changed, 6 insertions(+), 39 deletions(-) delete mode 100644 pulsaradmin/pkg/pulsar/common/errors.go diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index e55988dfa9..eaac5482da 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -359,7 +359,7 @@ func safeRespClose(resp *http.Response) { } } -// responseError is used to parse a response into a pulsar error +// responseError is used to parse a response into a client error func responseError(resp *http.Response) error { var e Error body, err := ioutil.ReadAll(resp.Body) @@ -369,7 +369,10 @@ func responseError(resp *http.Response) error { return e } - json.Unmarshal(body, &e) + err = json.Unmarshal(body, &e) + if err != nil { + e.Reason = string(body) + } e.Code = resp.StatusCode diff --git a/pulsaradmin/pkg/cli/errors.go b/pulsaradmin/pkg/cli/errors.go index 3a7e91a4c9..9a91465a98 100644 --- a/pulsaradmin/pkg/cli/errors.go +++ b/pulsaradmin/pkg/cli/errors.go @@ -19,7 +19,7 @@ package cli import "fmt" -const unknownErrorReason = "Unknown pulsar error" +const unknownErrorReason = "Unknown error" // Error is a admin error type type Error struct { diff --git a/pulsaradmin/pkg/pulsar/common/errors.go b/pulsaradmin/pkg/pulsar/common/errors.go deleted file mode 100644 index 191d936630..0000000000 --- a/pulsaradmin/pkg/pulsar/common/errors.go +++ /dev/null @@ -1,36 +0,0 @@ -// 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 common - -import "fmt" - -const UnknownErrorReason = "Unknown pulsar error" - -type Error struct { - Reason string `json:"reason"` - Code int -} - -func (e Error) Error() string { - return fmt.Sprintf("code: %d reason: %s", e.Code, e.Reason) -} - -func IsAdminError(err error) bool { - _, ok := err.(Error) - return ok -} From bf0f749635466ca9c6daa94be7b40f3736c3b8dd Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 20 Dec 2019 10:04:33 +0800 Subject: [PATCH 153/348] Fixes unexpected error when request using pulsar 2.4.2 proxy (streamnative/pulsarctl#157) *Motivation* When sending a GET or DELETE request to the Pulsar proxy, the server always threw an IllegalArgumentException. By my test that's because we are setting the Content-Type for a request without the request body. We remove the Content-Type if the request is GET or DELETE. *Modifications* - Remove the header Content-Type if the content body is empty. --- pulsaradmin/pkg/cli/client.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index eaac5482da..3f0bda7d83 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -65,10 +65,10 @@ func (c *Client) doRequest(r *request) (*http.Response, error) { if r.contentType != "" { req.Header.Set("Content-Type", r.contentType) - } else { - // add default headers + } else if req.Body != nil { req.Header.Set("Content-Type", "application/json") } + req.Header.Set("Accept", "application/json") req.Header.Set("User-Agent", c.useragent()) From b368faf091bbce7087b91852e81bb0da6a2874a8 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 6 Jan 2020 11:48:45 +0800 Subject: [PATCH 154/348] Fix wrong http header when receive pulsar header (streamnative/pulsarctl#165) --- Fixes streamnative/pulsarctl#164 *Motivation* When peekking a message using pulsarctl, the properties can not receive correctly. That's because the header is named as Camel when go http client receive the header. *Modifications* - Change the header prefix as camel --- pulsaradmin/pkg/pulsar/subscription.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/pulsar/subscription.go index 49a9531f39..9852aa9f97 100644 --- a/pulsaradmin/pkg/pulsar/subscription.go +++ b/pulsaradmin/pkg/pulsar/subscription.go @@ -181,7 +181,7 @@ func safeRespClose(resp *http.Response) { const ( PublishTimeHeader = "X-Pulsar-Publish-Time" BatchHeader = "X-Pulsar-Num-Batch-Message" - PropertyPrefix = "X-Pulsar-PROPERTY-" + PropertyPrefix = "X-Pulsar-Property-" ) func handleResp(topic utils.TopicName, resp *http.Response) ([]*utils.Message, error) { From 00f1a5d1b7c84ae21195fe6dc434d23dc167483f Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Sun, 12 Jan 2020 16:49:53 +0800 Subject: [PATCH 155/348] Using testcontainer to create a test framework for testing (streamnative/pulsarctl#167) * Using testcontainer to create a test framework for testing --- Master Issue: streamnative/pulsarctl#163 *Motivation* Currently, we are using an external pulsar standalone to test the Pulsarctl. We need to enhancement the test to ensure the Pulsarctl can work well in a complete pulsar cluster. *Modifications* - Using testcontainer to test the pulsarctl. * * Move direcotories * * Address comments --- pulsaradmin/go.mod | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index de596d5950..9f13bc2d1a 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -3,8 +3,8 @@ module github.com/streamnative/pulsar-admin-go go 1.12 require ( - github.com/davecgh/go-spew v1.1.1 github.com/dgrijalva/jwt-go v3.2.0+incompatible + github.com/docker/go-connections v0.4.0 github.com/fatih/color v1.7.0 // indirect github.com/golang/protobuf v1.3.1 github.com/google/go-cmp v0.3.1 // indirect @@ -16,7 +16,7 @@ require ( github.com/pkg/errors v0.8.1 github.com/spf13/cobra v0.0.5 github.com/spf13/pflag v1.0.3 - github.com/stretchr/objx v0.2.0 // indirect github.com/stretchr/testify v1.3.0 + github.com/testcontainers/testcontainers-go v0.0.10 gopkg.in/yaml.v2 v2.2.2 ) From cc95102e83a44d0094510bbde862119ffb34eead Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 13 Jan 2020 19:55:22 +0800 Subject: [PATCH 156/348] Add test for the auto recovery command (streamnative/pulsarctl#170) *Modifications* Add test for the auto-recovery command --- pulsaradmin/go.mod | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index 9f13bc2d1a..fba7d6e032 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -10,6 +10,7 @@ require ( github.com/google/go-cmp v0.3.1 // indirect github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b // indirect + github.com/magiconair/properties v1.8.0 github.com/mattn/go-colorable v0.1.2 // indirect github.com/mattn/go-runewidth v0.0.4 // indirect github.com/olekukonko/tablewriter v0.0.1 @@ -19,4 +20,5 @@ require ( github.com/stretchr/testify v1.3.0 github.com/testcontainers/testcontainers-go v0.0.10 gopkg.in/yaml.v2 v2.2.2 + gotest.tools v0.0.0-20181223230014-1083505acf35 ) From 019124ecfaf8bb657c01c936e9a34bfb70d98acb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Tue, 14 Jan 2020 17:04:59 +0800 Subject: [PATCH 157/348] Add context for pulsarctl (streamnative/pulsarctl#169) Signed-off-by: xiaolong.ran Master: streamnative/pulsarctl#119 - Add context for pulsarctl - pulsarctl context set [options] - pulsarctl context get - pulsarctl context rename [options] - pulsarctl context use [options] - pulsarctl context current - pulsarctl context delete [options] - add test case for context - remove `maligned` from `golangci.yml` - support as follows: - auth info - broker service url - bookie service url - add docs - how to use context --- pulsaradmin/go.mod | 11 +-- pulsaradmin/pkg/pulsar/utils/home_dir.go | 97 ++++++++++++++++++++++++ 2 files changed, 103 insertions(+), 5 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/utils/home_dir.go diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index fba7d6e032..b341f7754b 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -6,8 +6,8 @@ require ( github.com/dgrijalva/jwt-go v3.2.0+incompatible github.com/docker/go-connections v0.4.0 github.com/fatih/color v1.7.0 // indirect - github.com/golang/protobuf v1.3.1 - github.com/google/go-cmp v0.3.1 // indirect + github.com/golang/protobuf v1.3.2 + github.com/imdario/mergo v0.3.8 github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b // indirect github.com/magiconair/properties v1.8.0 @@ -16,9 +16,10 @@ require ( github.com/olekukonko/tablewriter v0.0.1 github.com/pkg/errors v0.8.1 github.com/spf13/cobra v0.0.5 - github.com/spf13/pflag v1.0.3 - github.com/stretchr/testify v1.3.0 + github.com/spf13/pflag v1.0.5 + github.com/stretchr/testify v1.4.0 github.com/testcontainers/testcontainers-go v0.0.10 - gopkg.in/yaml.v2 v2.2.2 + golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456 // indirect + gopkg.in/yaml.v2 v2.2.4 gotest.tools v0.0.0-20181223230014-1083505acf35 ) diff --git a/pulsaradmin/pkg/pulsar/utils/home_dir.go b/pulsaradmin/pkg/pulsar/utils/home_dir.go new file mode 100644 index 0000000000..330831c405 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/utils/home_dir.go @@ -0,0 +1,97 @@ +// 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 utils + +import ( + "os" + "path/filepath" + "runtime" +) + +// HomeDir returns the home directory for the current user. +// On Windows: +// 1. the first of %HOME%, %HOMEDRIVE%%HOMEPATH%, %USERPROFILE% containing a `.pulsar\config` file is returned. +// 2. if none of those locations contain a `.pulsar\config` file, the first of +// %HOME%, %USERPROFILE%, %HOMEDRIVE%%HOMEPATH% +// that exists and is writeable is returned. +// 3. if none of those locations are writeable, the first of %HOME%, %USERPROFILE%, %HOMEDRIVE%%HOMEPATH% +// that exists is returned. +// 4. if none of those locations exists, the first of %HOME%, %USERPROFILE%, %HOMEDRIVE%%HOMEPATH% +// that is set is returned. +func HomeDir() string { + if runtime.GOOS == "windows" { + home := os.Getenv("HOME") + homeDriveHomePath := "" + if homeDrive, homePath := os.Getenv("HOMEDRIVE"), os.Getenv("HOMEPATH"); len(homeDrive) > 0 && len(homePath) > 0 { + homeDriveHomePath = homeDrive + homePath + } + userProfile := os.Getenv("USERPROFILE") + + // Return first of %HOME%, %HOMEDRIVE%/%HOMEPATH%, %USERPROFILE% that contains a `.pulsar\config` file. + // %HOMEDRIVE%/%HOMEPATH% is preferred over %USERPROFILE% for backwards-compatibility. + for _, p := range []string{home, homeDriveHomePath, userProfile} { + if len(p) == 0 { + continue + } + if _, err := os.Stat(filepath.Join(p, ".pulsar", "config")); err != nil { + continue + } + return p + } + + firstSetPath := "" + firstExistingPath := "" + + // Prefer %USERPROFILE% over %HOMEDRIVE%/%HOMEPATH% for compatibility with other auth-writing tools + for _, p := range []string{home, userProfile, homeDriveHomePath} { + if len(p) == 0 { + continue + } + if len(firstSetPath) == 0 { + // remember the first path that is set + firstSetPath = p + } + info, err := os.Stat(p) + if err != nil { + continue + } + if len(firstExistingPath) == 0 { + // remember the first path that exists + firstExistingPath = p + } + if info.IsDir() && info.Mode().Perm()&(1<<(uint(7))) != 0 { + // return first path that is writeable + return p + } + } + + // If none are writeable, return first location that exists + if len(firstExistingPath) > 0 { + return firstExistingPath + } + + // If none exist, return first location that is set + if len(firstSetPath) > 0 { + return firstSetPath + } + + // We've got nothing + return "" + } + return os.Getenv("HOME") +} From 65c6c27b886d0d00a98049dd22e6e7d1f4794c8b Mon Sep 17 00:00:00 2001 From: Eron Wright Date: Tue, 14 Jan 2020 20:13:21 -0800 Subject: [PATCH 158/348] [issue-126] Implement JSON output format for list commands (streamnative/pulsarctl#181) * Add top-level output flag * Add output formats - add output flag (-o) - implement JSON output for list commands Closes streamnative/pulsarctl#126 Signed-off-by: Eron Wright * - register the output flags onto specific commands Signed-off-by: Eron Wright * - linting Signed-off-by: Eron Wright --- pulsaradmin/go.mod | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index b341f7754b..c4ee95a96b 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -6,11 +6,11 @@ require ( github.com/dgrijalva/jwt-go v3.2.0+incompatible github.com/docker/go-connections v0.4.0 github.com/fatih/color v1.7.0 // indirect + github.com/ghodss/yaml v1.0.0 github.com/golang/protobuf v1.3.2 github.com/imdario/mergo v0.3.8 github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b // indirect - github.com/magiconair/properties v1.8.0 github.com/mattn/go-colorable v0.1.2 // indirect github.com/mattn/go-runewidth v0.0.4 // indirect github.com/olekukonko/tablewriter v0.0.1 @@ -20,6 +20,6 @@ require ( github.com/stretchr/testify v1.4.0 github.com/testcontainers/testcontainers-go v0.0.10 golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456 // indirect + golang.org/x/tools v0.0.0-20200114191411-189207f339b7 // indirect gopkg.in/yaml.v2 v2.2.4 - gotest.tools v0.0.0-20181223230014-1083505acf35 ) From 9bd3f229271491471aa861ad39ea0f317fed9ff9 Mon Sep 17 00:00:00 2001 From: Nathan Mills Date: Sat, 18 Jan 2020 14:58:48 -0700 Subject: [PATCH 159/348] Rework auth and add pulsar client conf (streamnative/pulsarctl#196) Auth was reworked to allow custom Root CA without having to use TLS Auth. Changed the auth providers to wrap http.RoundTripper Added support for using the PULSAR_CLIENT_CONF env var for configuration like the java client does. --- pulsaradmin/go.mod | 3 +- pulsaradmin/pkg/auth/auth_provider.go | 65 ++++++++++++- pulsaradmin/pkg/auth/tls.go | 93 +++++++++---------- pulsaradmin/pkg/auth/token.go | 81 ++++++++-------- pulsaradmin/pkg/cli/client.go | 22 +---- pulsaradmin/pkg/pulsar/admin.go | 58 ++---------- .../{admin_config.go => common/config.go} | 49 ++++------ pulsaradmin/pkg/pulsar/defaults.go | 28 ++++++ 8 files changed, 203 insertions(+), 196 deletions(-) rename pulsaradmin/pkg/pulsar/{admin_config.go => common/config.go} (57%) create mode 100644 pulsaradmin/pkg/pulsar/defaults.go diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index c4ee95a96b..e2aeb6340a 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -11,6 +11,7 @@ require ( github.com/imdario/mergo v0.3.8 github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b // indirect + github.com/magiconair/properties v1.8.0 github.com/mattn/go-colorable v0.1.2 // indirect github.com/mattn/go-runewidth v0.0.4 // indirect github.com/olekukonko/tablewriter v0.0.1 @@ -20,6 +21,6 @@ require ( github.com/stretchr/testify v1.4.0 github.com/testcontainers/testcontainers-go v0.0.10 golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456 // indirect - golang.org/x/tools v0.0.0-20200114191411-189207f339b7 // indirect + golang.org/x/tools v0.0.0-20200116203608-1c4842a210a7 // indirect gopkg.in/yaml.v2 v2.2.4 ) diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index f4d595ea8c..308dd9aec0 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -17,11 +17,68 @@ package auth +import ( + "crypto/tls" + "crypto/x509" + "fmt" + "io/ioutil" + "net/http" + "os" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" +) + // Provider provide a general method to add auth message type Provider interface { - // HasDataForHTTP is used to check if data for HTTP are available - HasDataForHTTP() bool + RoundTrip(req *http.Request) (*http.Response, error) + Transport() http.RoundTripper +} + +type Transport struct { + T http.RoundTripper +} + +func GetAuthProvider(config *common.Config) (*Provider, error) { + var provider Provider + defaultTransport := getDefaultTransport(config) + var err error + switch config.AuthPlugin { + case TLSPluginName: + provider, err = NewAuthenticationTLSFromAuthParams(config.AuthParams, defaultTransport) + case TokenPluginName: + provider, err = NewAuthenticationTokenFromAuthParams(config.AuthParams, defaultTransport) + default: + switch { + case len(config.TLSCertFile) > 0 && len(config.TLSKeyFile) > 0: + provider, err = NewAuthenticationTLS(config.TLSCertFile, config.TLSKeyFile, defaultTransport) + case len(config.Token) > 0: + provider, err = NewAuthenticationToken(config.Token, defaultTransport) + case len(config.TokenFile) > 0: + provider, err = NewAuthenticationTokenFromFile(config.TokenFile, defaultTransport) + } + } + return &provider, err +} + +func getDefaultTransport(config *common.Config) http.RoundTripper { + transport := http.DefaultTransport.(*http.Transport) + if !config.TLSAllowInsecureConnection && len(config.TLSTrustCertsFilePath) == 0 { + return transport + } - // GetHTTPHeaders is used to get all auth headers - GetHTTPHeaders() (map[string]string, error) + tlsConfig := &tls.Config{ + InsecureSkipVerify: config.TLSAllowInsecureConnection, + } + if len(config.TLSTrustCertsFilePath) > 0 { + rootCA, err := ioutil.ReadFile(config.TLSTrustCertsFilePath) + if err != nil { + fmt.Fprintln(os.Stderr, "error loading certificate authority:", err) + os.Exit(1) + } + tlsConfig.RootCAs = x509.NewCertPool() + tlsConfig.RootCAs.AppendCertsFromPEM(rootCA) + } + transport.MaxIdleConnsPerHost = 10 + transport.TLSClientConfig = tlsConfig + return transport } diff --git a/pulsaradmin/pkg/auth/tls.go b/pulsaradmin/pkg/auth/tls.go index 1f2a406f26..2cad869d00 100644 --- a/pulsaradmin/pkg/auth/tls.go +++ b/pulsaradmin/pkg/auth/tls.go @@ -19,37 +19,49 @@ package auth import ( "crypto/tls" - "crypto/x509" - "io/ioutil" + "net/http" + "strings" +) - "github.com/pkg/errors" +const ( + TLSPluginName = "org.apache.pulsar.client.impl.auth.AuthenticationTls" ) type TLSAuthProvider struct { - certificatePath string - privateKeyPath string - allowInsecureConnection bool + certificatePath string + privateKeyPath string + T http.RoundTripper } // NewAuthenticationTLS initialize the authentication provider func NewAuthenticationTLS(certificatePath string, privateKeyPath string, - allowInsecureConnection bool) *TLSAuthProvider { - - return &TLSAuthProvider{ - certificatePath: certificatePath, - privateKeyPath: privateKeyPath, - allowInsecureConnection: allowInsecureConnection, + transport http.RoundTripper) (*TLSAuthProvider, error) { + provider := &TLSAuthProvider{ + certificatePath: certificatePath, + privateKeyPath: privateKeyPath, + T: transport, } + if err := provider.configTLS(); err != nil { + return nil, err + } + return provider, nil } -func (p *TLSAuthProvider) Init() error { - // Try to read certificates immediately to provide better error at startup - _, err := p.GetTLSCertificate() - return err -} - -func (p *TLSAuthProvider) Name() string { - return "tls" +func NewAuthenticationTLSFromAuthParams(encodedAuthParams string, + transport http.RoundTripper) (*TLSAuthProvider, error) { + var certificatePath string + var privateKeyPath string + parts := strings.Split(encodedAuthParams, ",") + for _, part := range parts { + kv := strings.Split(part, ":") + switch kv[0] { + case "tlsCertFile": + certificatePath = kv[1] + case "tlsKeyFile": + privateKeyPath = kv[1] + } + } + return NewAuthenticationTLS(certificatePath, privateKeyPath, transport) } func (p *TLSAuthProvider) GetTLSCertificate() (*tls.Certificate, error) { @@ -57,39 +69,20 @@ func (p *TLSAuthProvider) GetTLSCertificate() (*tls.Certificate, error) { return &cert, err } -func (p *TLSAuthProvider) GetTLSConfig(certFile string, allowInsecureConnection bool) (*tls.Config, error) { - tlsConfig := &tls.Config{ - InsecureSkipVerify: allowInsecureConnection, - } - - if certFile != "" { - caCerts, err := ioutil.ReadFile(certFile) - if err != nil { - return nil, err - } +func (p *TLSAuthProvider) RoundTrip(req *http.Request) (*http.Response, error) { + return p.T.RoundTrip(req) +} - tlsConfig.RootCAs = x509.NewCertPool() - if !tlsConfig.RootCAs.AppendCertsFromPEM(caCerts) { - return nil, errors.New("failed to parse root CAs certificates") - } - } +func (p *TLSAuthProvider) Transport() http.RoundTripper { + return p.T +} +func (p *TLSAuthProvider) configTLS() error { cert, err := p.GetTLSCertificate() if err != nil { - return nil, err - } - - if cert != nil { - tlsConfig.Certificates = []tls.Certificate{*cert} + return err } - - return tlsConfig, nil -} - -func (p *TLSAuthProvider) HasDataForHTTP() bool { - return false -} - -func (p *TLSAuthProvider) GetHTTPHeaders() (map[string]string, error) { - return nil, errors.New("Unsupported operation") + transport := p.T.(*http.Transport) + transport.TLSClientConfig.Certificates = []tls.Certificate{*cert} + return nil } diff --git a/pulsaradmin/pkg/auth/token.go b/pulsaradmin/pkg/auth/token.go index 008b1825b7..b825f669f0 100644 --- a/pulsaradmin/pkg/auth/token.go +++ b/pulsaradmin/pkg/auth/token.go @@ -18,70 +18,63 @@ package auth import ( + "fmt" "io/ioutil" + "net/http" "strings" "github.com/pkg/errors" ) +const ( + tokenPrefix = "token:" + filePrefix = "file:" + TokenPluginName = "org.apache.pulsar.client.impl.auth.AuthenticationToken" +) + type TokenAuthProvider struct { - tokenSupplier func() (string, error) + T http.RoundTripper + token string } // NewAuthenticationToken return a interface of Provider with a string token. -func NewAuthenticationToken(token string) *TokenAuthProvider { - return &TokenAuthProvider{ - tokenSupplier: func() (string, error) { - if token == "" { - return "", errors.New("empty token credentials") - } - return token, nil - }, +func NewAuthenticationToken(token string, transport http.RoundTripper) (*TokenAuthProvider, error) { + if len(token) == 0 { + return nil, errors.New("No token provided") } + return &TokenAuthProvider{token: token, T: transport}, nil } // NewAuthenticationTokenFromFile return a interface of a Provider with a string token file path. -func NewAuthenticationTokenFromFile(tokenFilePath string) *TokenAuthProvider { - return &TokenAuthProvider{ - tokenSupplier: func() (string, error) { - data, err := ioutil.ReadFile(tokenFilePath) - if err != nil { - return "", err - } - - token := strings.Trim(string(data), " \n") - if token == "" { - return "", errors.New("empty token credentials") - } - return token, nil - }, +func NewAuthenticationTokenFromFile(tokenFilePath string, transport http.RoundTripper) (*TokenAuthProvider, error) { + data, err := ioutil.ReadFile(tokenFilePath) + if err != nil { + return nil, err } + token := strings.Trim(string(data), " \n") + return NewAuthenticationToken(token, transport) } -func (p *TokenAuthProvider) Init() error { - // Try to read certificates immediately to provide better error at startup - _, err := p.GetData() - return err -} - -func (p *TokenAuthProvider) GetData() ([]byte, error) { - t, err := p.tokenSupplier() - if err != nil { - return nil, err +func NewAuthenticationTokenFromAuthParams(encodedAuthParam string, + transport http.RoundTripper) (*TokenAuthProvider, error) { + var tokenAuthProvider *TokenAuthProvider + var err error + switch { + case strings.HasPrefix(encodedAuthParam, tokenPrefix): + tokenAuthProvider, err = NewAuthenticationToken(strings.TrimPrefix(encodedAuthParam, tokenPrefix), transport) + case strings.HasPrefix(encodedAuthParam, filePrefix): + tokenAuthProvider, err = NewAuthenticationTokenFromFile(strings.TrimPrefix(encodedAuthParam, filePrefix), transport) + default: + tokenAuthProvider, err = NewAuthenticationToken(encodedAuthParam, transport) } - return []byte(t), nil + return tokenAuthProvider, err } -func (p *TokenAuthProvider) HasDataForHTTP() bool { - return true +func (p *TokenAuthProvider) RoundTrip(req *http.Request) (*http.Response, error) { + req.Header.Add("Authorization", fmt.Sprintf("Bearer %s", p.token)) + return p.T.RoundTrip(req) } -func (p *TokenAuthProvider) GetHTTPHeaders() (map[string]string, error) { - data, err := p.GetData() - if err != nil { - return nil, err - } - headers := make(map[string]string) - headers["Authorization"] = "Bearer " + string(data) - return headers, nil +func (p *TokenAuthProvider) Transport() http.RoundTripper { + return p.T } diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index 3f0bda7d83..30d10fe89f 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -25,16 +25,13 @@ import ( "net/http" "net/url" "path" - - "github.com/streamnative/pulsar-admin-go/pkg/auth" ) // Client is a base client that is used to make http request to the ServiceURL type Client struct { - ServiceURL string - HTTPClient *http.Client - VersionInfo string - AuthProvider auth.Provider + ServiceURL string + HTTPClient *http.Client + VersionInfo string } func (c *Client) newRequest(method, path string) (*request, error) { @@ -71,19 +68,6 @@ func (c *Client) doRequest(r *request) (*http.Response, error) { req.Header.Set("Accept", "application/json") req.Header.Set("User-Agent", c.useragent()) - - if c.AuthProvider != nil { - if c.AuthProvider.HasDataForHTTP() { - headers, err := c.AuthProvider.GetHTTPHeaders() - if err != nil { - return nil, err - } - for k, v := range headers { - req.Header.Set(k, v) - } - } - } - hc := c.HTTPClient if hc == nil { hc = http.DefaultClient diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 57772a6c7b..d13a67b036 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -18,9 +18,9 @@ package pulsar import ( + "fmt" "net/http" "path" - "strings" "github.com/streamnative/pulsar-admin-go/pkg/auth" "github.com/streamnative/pulsar-admin-go/pkg/cli" @@ -58,13 +58,13 @@ type pulsarClient struct { } // New returns a new client -func New(config *Config) (Client, error) { +func New(config *common.Config) (Client, error) { if len(config.WebServiceURL) == 0 { config.WebServiceURL = DefaultWebServiceURL } c := &pulsarClient{ - APIVersion: config.APIVersion, + APIVersion: config.PulsarAPIVersion, Client: &cli.Client{ ServiceURL: config.WebServiceURL, VersionInfo: ReleaseVersion, @@ -74,52 +74,14 @@ func New(config *Config) (Client, error) { }, } - err := c.initAuth(config) - if err != nil { - return nil, err - } - - return c, nil -} - -func (c *pulsarClient) initAuth(config *Config) error { - if strings.HasPrefix(config.WebServiceURL, "https") { - err := c.initTLS(config) - if err != nil { - return err - } - } - - if config.TokenFile != "" || config.Token != "" { - c.initToken(config) - } - - return nil -} - -func (c *pulsarClient) initTLS(config *Config) error { - tlsAuth := auth.NewAuthenticationTLS(config.TLSCertFile, config.TLSKeyFile, config.TLSAllowInsecureConnection) - tlsConf, err := tlsAuth.GetTLSConfig(config.TLSCertFile, config.TLSAllowInsecureConnection) - if err != nil { - return err - } - - c.Client.HTTPClient.Transport = &http.Transport{ - MaxIdleConnsPerHost: 10, - TLSClientConfig: tlsConf, - } - - return nil -} - -func (c *pulsarClient) initToken(config *Config) { - if config.Token != "" { - c.Client.AuthProvider = auth.NewAuthenticationToken(config.Token) - } - - if config.TokenFile != "" { - c.Client.AuthProvider = auth.NewAuthenticationTokenFromFile(config.TokenFile) + authProvider, err := auth.GetAuthProvider(config) + if authProvider != nil { + fmt.Printf("Found Auth provider %T", authProvider) + c.Client.HTTPClient.Transport = *authProvider + } else { + fmt.Printf("No Auth Provider found") } + return c, err } func (c *pulsarClient) endpoint(componentPath string, parts ...string) string { diff --git a/pulsaradmin/pkg/pulsar/admin_config.go b/pulsaradmin/pkg/pulsar/common/config.go similarity index 57% rename from pulsaradmin/pkg/pulsar/admin_config.go rename to pulsaradmin/pkg/pulsar/common/config.go index ff4bdcf771..ca02fef554 100644 --- a/pulsaradmin/pkg/pulsar/admin_config.go +++ b/pulsaradmin/pkg/pulsar/common/config.go @@ -15,42 +15,31 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package common -import ( - "time" +type Config struct { + // the web service url that pulsarctl connects to. Default is http://localhost:8080 + WebServiceURL string - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" -) + // the bookkeeper service url that pulsarctl connects to. + BKWebServiceURL string + // Set the path to the trusted TLS certificate file + TLSTrustCertsFilePath string + // Configure whether the Pulsar client accept untrusted TLS certificate from broker (default: false) + TLSAllowInsecureConnection bool -const ( - DefaultWebServiceURL = "http://localhost:8080" - DefaultHTTPTimeOutDuration = 5 * time.Minute -) + TLSEnableHostnameVerification bool -var ReleaseVersion = "None" + AuthPlugin string -// Config is used to configure the admin client -type Config struct { - WebServiceURL string - HTTPTimeout time.Duration - // TODO: api version should apply to the method - APIVersion common.APIVersion + AuthParams string - //Auth *auth.TLSAuthProvider - TLSCertFile string - TLSKeyFile string - TLSAllowInsecureConnection bool + // TLS Cert and Key Files for authentication + TLSCertFile string + TLSKeyFile string // Token and TokenFile is used to config the pulsarctl using token to authentication - Token string - TokenFile string -} - -// DefaultConfig returns a default configuration for the pulsar admin client -func DefaultConfig() *Config { - config := &Config{ - WebServiceURL: DefaultWebServiceURL, - } - return config + Token string + TokenFile string + PulsarAPIVersion APIVersion } diff --git a/pulsaradmin/pkg/pulsar/defaults.go b/pulsaradmin/pkg/pulsar/defaults.go new file mode 100644 index 0000000000..50e67bcb32 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/defaults.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 pulsar + +import ( + "time" +) + +const ( + DefaultWebServiceURL = "http://localhost:8080" + DefaultHTTPTimeOutDuration = 5 * time.Minute + ReleaseVersion = "None" +) From 0ee48325034b317a642c41a5d6541597bad32083 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Mon, 20 Jan 2020 13:41:41 +0800 Subject: [PATCH 160/348] [issue:197]Check Nil interface the right way (streamnative/pulsarctl#198) Signed-off-by: xiaolong.ran Fixes: streamnative/pulsarctl#197 - Check Nil interface --- pulsaradmin/pkg/pulsar/admin.go | 6 +-- pulsaradmin/pkg/pulsar/utils/utils.go | 12 +++++ pulsaradmin/pkg/pulsar/utils/utils_test.go | 61 ++++++++++++++++++++++ 3 files changed, 76 insertions(+), 3 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/utils/utils_test.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index d13a67b036..c4a518a1f9 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -75,11 +75,11 @@ func New(config *common.Config) (Client, error) { } authProvider, err := auth.GetAuthProvider(config) - if authProvider != nil { - fmt.Printf("Found Auth provider %T", authProvider) + if !utils.IsNilFixed(authProvider) { + fmt.Printf("Found Auth provider %T\n", authProvider) c.Client.HTTPClient.Transport = *authProvider } else { - fmt.Printf("No Auth Provider found") + fmt.Printf("No Auth Provider found\n") } return c, err } diff --git a/pulsaradmin/pkg/pulsar/utils/utils.go b/pulsaradmin/pkg/pulsar/utils/utils.go index e371b6a151..7b23b15654 100644 --- a/pulsaradmin/pkg/pulsar/utils/utils.go +++ b/pulsaradmin/pkg/pulsar/utils/utils.go @@ -19,8 +19,20 @@ package utils import ( "fmt" + "reflect" ) func MakeHTTPPath(apiVersion string, componentPath string) string { return fmt.Sprintf("/admin/%s%s", apiVersion, componentPath) } + +func IsNilFixed(i interface{}) bool { + if i == nil { + return true + } + switch reflect.TypeOf(i).Kind() { + case reflect.Ptr, reflect.Map, reflect.Array, reflect.Chan, reflect.Slice: + return reflect.ValueOf(i).IsNil() + } + return false +} diff --git a/pulsaradmin/pkg/pulsar/utils/utils_test.go b/pulsaradmin/pkg/pulsar/utils/utils_test.go new file mode 100644 index 0000000000..b9ce770aed --- /dev/null +++ b/pulsaradmin/pkg/pulsar/utils/utils_test.go @@ -0,0 +1,61 @@ +// 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 utils + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +type People interface { + MakeSound() string +} + +type Student struct{} + +func (s *Student) MakeSound() string { + return "Student" +} + +type Teacher struct{} + +func (t Teacher) MakeSound() string { + return "Teacher" +} + +//nolint +func TestIsNilFixed(t *testing.T) { + var stu *Student = nil + var people People + people = stu + + var teacher Teacher + people = teacher + + assert.False(t, IsNilFixed(people)) + + var m map[string]string + assert.True(t, IsNilFixed(m)) + + var s []string + assert.True(t, IsNilFixed(s)) + + var ch chan string + assert.True(t, IsNilFixed(ch)) +} From 46f9c69bf4f271a325f392c2d2347bcb204bbce0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Mon, 20 Jan 2020 17:16:29 +0800 Subject: [PATCH 161/348] [issue:182] get-function-assignments has incorrect output (streamnative/pulsarctl#199) Signed-off-by: xiaolong.ran get-function-assignments has incorrect output --- pulsaradmin/pkg/pulsar/admin.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index c4a518a1f9..e2b75509f0 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -76,7 +76,6 @@ func New(config *common.Config) (Client, error) { authProvider, err := auth.GetAuthProvider(config) if !utils.IsNilFixed(authProvider) { - fmt.Printf("Found Auth provider %T\n", authProvider) c.Client.HTTPClient.Transport = *authProvider } else { fmt.Printf("No Auth Provider found\n") From 96d284a78f253965855b31fe50f8847a84f7c7ad Mon Sep 17 00:00:00 2001 From: Nathan Mills Date: Wed, 22 Jan 2020 02:30:52 -0700 Subject: [PATCH 162/348] Change NameSpaceAuth to be an [] of AuthAction (streamnative/pulsarctl#202) * Change NameSpaceAuth to be an [] of AuthAction * Format auth_polices --- pulsaradmin/pkg/pulsar/common/auth_polices.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/common/auth_polices.go b/pulsaradmin/pkg/pulsar/common/auth_polices.go index dd6dee82bf..5f5dfe4df0 100644 --- a/pulsaradmin/pkg/pulsar/common/auth_polices.go +++ b/pulsaradmin/pkg/pulsar/common/auth_polices.go @@ -18,14 +18,14 @@ package common type AuthPolicies struct { - NamespaceAuth map[string]AuthAction `json:"namespace_auth"` + NamespaceAuth map[string][]AuthAction `json:"namespace_auth"` DestinationAuth map[string]map[string]AuthAction `json:"destination_auth"` SubscriptionAuthRoles map[string][]string `json:"subscription_auth_roles"` } func NewAuthPolicies() *AuthPolicies { return &AuthPolicies{ - NamespaceAuth: make(map[string]AuthAction), + NamespaceAuth: make(map[string][]AuthAction), DestinationAuth: make(map[string]map[string]AuthAction), SubscriptionAuthRoles: make(map[string][]string), } From 1441aa09980c47715d1bf23a2a665864b4410e52 Mon Sep 17 00:00:00 2001 From: Nathan Mills Date: Wed, 22 Jan 2020 02:32:16 -0700 Subject: [PATCH 163/348] Add RetentionPolicy parsing (streamnative/pulsarctl#201) --- pulsaradmin/pkg/pulsar/utils/backlog_quota.go | 17 +++++++++++++++++ pulsaradmin/pkg/pulsar/utils/namespace_name.go | 6 +++--- 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/utils/backlog_quota.go b/pulsaradmin/pkg/pulsar/utils/backlog_quota.go index 17139e7aad..ccfce493c4 100644 --- a/pulsaradmin/pkg/pulsar/utils/backlog_quota.go +++ b/pulsaradmin/pkg/pulsar/utils/backlog_quota.go @@ -17,6 +17,8 @@ package utils +import "github.com/pkg/errors" + type BacklogQuota struct { Limit int64 `json:"limit"` Policy RetentionPolicy `json:"policy"` @@ -40,3 +42,18 @@ const ( ProducerException RetentionPolicy = "producer_exception" ConsumerBacklogEviction RetentionPolicy = "consumer_backlog_eviction" ) + +func ParseRetentionPolicy(str string) (RetentionPolicy, error) { + switch str { + case ProducerException.String(): + return ProducerException, nil + case ConsumerBacklogEviction.String(): + return ConsumerBacklogEviction, nil + default: + return "", errors.Errorf("Invalid retention policy %s", str) + } +} + +func (s RetentionPolicy) String() string { + return string(s) +} diff --git a/pulsaradmin/pkg/pulsar/utils/namespace_name.go b/pulsaradmin/pkg/pulsar/utils/namespace_name.go index fbf36acd7f..f35c66eef0 100644 --- a/pulsaradmin/pkg/pulsar/utils/namespace_name.go +++ b/pulsaradmin/pkg/pulsar/utils/namespace_name.go @@ -65,12 +65,12 @@ func validateNamespaceName(tenant, namespace string) error { return errors.Errorf("Invalid tenant or namespace. [%s/%s]", tenant, namespace) } - ok := checkName(tenant) + ok := CheckName(tenant) if !ok { return errors.Errorf("Tenant name include unsupported special chars. tenant : [%s]", tenant) } - ok = checkName(namespace) + ok = CheckName(namespace) if !ok { return errors.Errorf("Namespace name include unsupported special chars. namespace : [%s]", namespace) } @@ -83,7 +83,7 @@ func validateNamespaceName(tenant, namespace string) error { // and % is allowed as part of valid URL encoding const PATTEN = "^[-=:.\\w]*$" -func checkName(name string) bool { +func CheckName(name string) bool { patten, err := regexp.Compile(PATTEN) if err != nil { return false From ecb95e702ab85b7d14818342a7ef793363b96ac3 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 5 Feb 2020 09:48:15 +0800 Subject: [PATCH 164/348] [Issue 183] Fixes can not parse the `destination_auth` (streamnative/pulsarctl#204) Fixes streamnative/pulsarctl#183 *Modifications* Make the `destination_auth` receives a auth action array. --- pulsaradmin/pkg/pulsar/common/auth_polices.go | 8 +-- .../pkg/pulsar/common/auth_polices_test.go | 65 +++++++++++++++++++ 2 files changed, 69 insertions(+), 4 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/common/auth_polices_test.go diff --git a/pulsaradmin/pkg/pulsar/common/auth_polices.go b/pulsaradmin/pkg/pulsar/common/auth_polices.go index 5f5dfe4df0..ed97b3fbd6 100644 --- a/pulsaradmin/pkg/pulsar/common/auth_polices.go +++ b/pulsaradmin/pkg/pulsar/common/auth_polices.go @@ -18,15 +18,15 @@ package common type AuthPolicies struct { - NamespaceAuth map[string][]AuthAction `json:"namespace_auth"` - DestinationAuth map[string]map[string]AuthAction `json:"destination_auth"` - SubscriptionAuthRoles map[string][]string `json:"subscription_auth_roles"` + NamespaceAuth map[string][]AuthAction `json:"namespace_auth"` + DestinationAuth map[string]map[string][]AuthAction `json:"destination_auth"` + SubscriptionAuthRoles map[string][]string `json:"subscription_auth_roles"` } func NewAuthPolicies() *AuthPolicies { return &AuthPolicies{ NamespaceAuth: make(map[string][]AuthAction), - DestinationAuth: make(map[string]map[string]AuthAction), + DestinationAuth: make(map[string]map[string][]AuthAction), SubscriptionAuthRoles: make(map[string][]string), } } diff --git a/pulsaradmin/pkg/pulsar/common/auth_polices_test.go b/pulsaradmin/pkg/pulsar/common/auth_polices_test.go new file mode 100644 index 0000000000..37a39ce05c --- /dev/null +++ b/pulsaradmin/pkg/pulsar/common/auth_polices_test.go @@ -0,0 +1,65 @@ +// 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 common + +import ( + "encoding/json" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestAuthPolicies(t *testing.T) { + testData := "{\n" + + " \"namespace_auth\": {\n" + + " \"persistent://public/default/ns_auth\": [\n" + + " \"produce\",\n" + + " \"consume\",\n" + + " \"function\"\n" + + " ]\n" + + " },\n" + + " \"destination_auth\": {\n" + + " \"persistent://public/default/dest_auth\": {\n" + + " \"admin-role\": [\n" + + " \"produce\",\n" + + " \"consume\",\n" + + " \"function\"\n" + + " ]\n" + + " },\n" + + " \"persistent://public/default/dest_auth_1\": {\n" + + " \"grant-partitioned-role\": [\n" + + " \"produce\",\n" + + " \"consume\"\n" + + " ]\n" + + " },\n" + + " \"persistent://public/default/test-revoke-partitioned-topic\": {},\n" + + " \"persistent://public/default/test-revoke-non-partitioned-topic\": {}\n },\n" + + " \"subscription_auth_roles\": {}\n" + + "}" + + policies := &AuthPolicies{} + err := json.Unmarshal([]byte(testData), policies) + if err != nil { + t.Fatal(err) + } + + assert.Equal(t, 3, len(policies.NamespaceAuth["persistent://public/default/ns_auth"])) + assert.Equal(t, 4, len(policies.DestinationAuth)) + assert.Equal(t, 3, len(policies.DestinationAuth["persistent://public/default/dest_auth"]["admin-role"])) + assert.Equal(t, 0, len(policies.SubscriptionAuthRoles)) +} From aeda3fef7820025a98a39fce8785390c893eccf5 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 11 Feb 2020 12:10:00 +0800 Subject: [PATCH 165/348] [Issue 180] Make error log use stderr (streamnative/pulsarctl#206) * [Issue 180] Make error log use stderr --- Fixes streamnative/pulsarctl#180 *Motivation* Using error handler to handle all commands error, and use stderr to output error messages. --- pulsaradmin/go.mod | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index e2aeb6340a..e2980e044a 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -5,12 +5,12 @@ go 1.12 require ( github.com/dgrijalva/jwt-go v3.2.0+incompatible github.com/docker/go-connections v0.4.0 - github.com/fatih/color v1.7.0 // indirect + github.com/fatih/color v1.7.0 github.com/ghodss/yaml v1.0.0 github.com/golang/protobuf v1.3.2 github.com/imdario/mergo v0.3.8 github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 - github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b // indirect + github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b github.com/magiconair/properties v1.8.0 github.com/mattn/go-colorable v0.1.2 // indirect github.com/mattn/go-runewidth v0.0.4 // indirect From e06dc2272aa08944cff40efc3d9aaf2d0543d710 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=89=E5=B0=8F=E9=BE=99?= Date: Thu, 2 Apr 2020 07:09:35 +0800 Subject: [PATCH 166/348] Add download command for functions CLI (streamnative/pulsarctl#221) Signed-off-by: xiaolong.ran --- pulsaradmin/go.mod | 1 + pulsaradmin/pkg/pulsar/functions.go | 58 ++++++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/utils/data.go | 2 + 3 files changed, 61 insertions(+) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index e2980e044a..24967d24df 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -3,6 +3,7 @@ module github.com/streamnative/pulsar-admin-go go 1.12 require ( + github.com/davecgh/go-spew v1.1.1 github.com/dgrijalva/jwt-go v3.2.0+incompatible github.com/docker/go-connections v0.4.0 github.com/fatih/color v1.7.0 diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index f9ac0c9be1..ee7d168ba4 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -57,6 +57,24 @@ type Functions interface { // DeleteFunction delete an existing function DeleteFunction(tenant, namespace, name string) error + // Download Function Code + // @param destinationFile + // file where data should be downloaded to + // @param path + // Path where data is located + DownloadFunction(path, destinationFile string) error + + // Download Function Code + // @param destinationFile + // file where data should be downloaded to + // @param tenant + // Tenant name + // @param namespace + // Namespace name + // @param function + // Function name + DownloadFunctionByNs(destinationFile, tenant, namespace, function string) error + // StartFunction start all function instances StartFunction(tenant, namespace, name string) error @@ -260,6 +278,46 @@ func (f *functions) DeleteFunction(tenant, namespace, name string) error { return f.pulsar.Client.Delete(endpoint) } +func (f *functions) DownloadFunction(path, destinationFile string) error { + endpoint := f.pulsar.endpoint(f.basePath, "download") + _, err := os.Open(destinationFile) + if err != nil { + _, err = os.Create(destinationFile) + if err != nil { + return err + } + } + + tmpMap := make(map[string]string) + tmpMap["path"] = path + + _, err = f.pulsar.Client.GetWithQueryParams(endpoint, nil, tmpMap, false) + + if err != nil { + return err + } + + return nil +} + +func (f *functions) DownloadFunctionByNs(destinationFile, tenant, namespace, function string) error { + endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, function, "download") + _, err := os.Open(destinationFile) + if err != nil { + _, err = os.Create(destinationFile) + if err != nil { + return err + } + } + + err = f.pulsar.Client.Get(endpoint, nil) + if err != nil { + return err + } + + return nil +} + func (f *functions) StartFunction(tenant, namespace, name string) error { endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name) return f.pulsar.Client.Post(endpoint+"/start", "") diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index ab0ae08026..2021642d30 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -73,6 +73,8 @@ type FunctionData struct { FunctionConfigFile string `json:"fnConfigFile"` ProcessingGuarantees string `json:"processingGuarantees"` UserConfig string `json:"userConfigString"` + DestinationFile string `json:"destinationFile"` + Path string `json:"path"` FuncConf *FunctionConfig `json:"-"` } From 2827b6d6bd991d3d32d808c2a6fe9612ed342919 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 23 Jun 2020 12:32:39 +0800 Subject: [PATCH 167/348] Fixes nil pointer when using tls (streamnative/pulsarctl#231) --- Fixes streamnative/pulsarctl#230 *Motivation* The tlsconfig in default transport always needs to configure with arguments. --- pulsaradmin/pkg/auth/auth_provider.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index 308dd9aec0..f62cba857e 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -62,10 +62,6 @@ func GetAuthProvider(config *common.Config) (*Provider, error) { func getDefaultTransport(config *common.Config) http.RoundTripper { transport := http.DefaultTransport.(*http.Transport) - if !config.TLSAllowInsecureConnection && len(config.TLSTrustCertsFilePath) == 0 { - return transport - } - tlsConfig := &tls.Config{ InsecureSkipVerify: config.TLSAllowInsecureConnection, } From 88f9f0ef548078f8cc1022231bb4ee70026150ab Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 7 Jul 2020 10:06:52 +0800 Subject: [PATCH 168/348] Support oauth2 authentication for pulsarctl (streamnative/pulsarctl#233) Support oauth2 client credential authentication flow for pulsarctl. The pulsarctl will get the access token from an Authorization service to access protected pulsar resource. --- pulsaradmin/go.mod | 11 +- pulsaradmin/pkg/auth/auth_provider.go | 7 + pulsaradmin/pkg/auth/oauth2.go | 94 +++++ pulsaradmin/pkg/auth/oauth2/auth.go | 88 +++++ .../pkg/auth/oauth2/auth_suite_test.go | 59 +++ .../oauth2/authorization_tokenretriever.go | 338 ++++++++++++++++++ .../authorization_tokenretriever_test.go | 334 +++++++++++++++++ .../auth/oauth2/client_credentials_flow.go | 149 ++++++++ .../oauth2/client_credentials_flow_test.go | 135 +++++++ .../oauth2/client_credentials_provider.go | 65 ++++ .../pkg/auth/oauth2/config_tokenprovider.go | 57 +++ .../auth/oauth2/config_tokenprovider_test.go | 91 +++++ .../pkg/auth/oauth2/oidc_endpoint_provider.go | 58 +++ .../oauth2/oidc_endpoint_provider_test.go | 92 +++++ pulsaradmin/pkg/auth/oauth2/plugin/clock.go | 99 +++++ .../pkg/auth/oauth2/plugin/fake_clock.go | 270 ++++++++++++++ pulsaradmin/pkg/auth/oauth2_test.go | 106 ++++++ pulsaradmin/pkg/pulsar/common/config.go | 6 + 18 files changed, 2055 insertions(+), 4 deletions(-) create mode 100644 pulsaradmin/pkg/auth/oauth2.go create mode 100644 pulsaradmin/pkg/auth/oauth2/auth.go create mode 100644 pulsaradmin/pkg/auth/oauth2/auth_suite_test.go create mode 100644 pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever.go create mode 100644 pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever_test.go create mode 100644 pulsaradmin/pkg/auth/oauth2/client_credentials_flow.go create mode 100644 pulsaradmin/pkg/auth/oauth2/client_credentials_flow_test.go create mode 100644 pulsaradmin/pkg/auth/oauth2/client_credentials_provider.go create mode 100644 pulsaradmin/pkg/auth/oauth2/config_tokenprovider.go create mode 100644 pulsaradmin/pkg/auth/oauth2/config_tokenprovider_test.go create mode 100644 pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider.go create mode 100644 pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider_test.go create mode 100644 pulsaradmin/pkg/auth/oauth2/plugin/clock.go create mode 100644 pulsaradmin/pkg/auth/oauth2/plugin/fake_clock.go create mode 100644 pulsaradmin/pkg/auth/oauth2_test.go diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index 24967d24df..d1665030f9 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -3,7 +3,7 @@ module github.com/streamnative/pulsar-admin-go go 1.12 require ( - github.com/davecgh/go-spew v1.1.1 + github.com/99designs/keyring v1.1.5 github.com/dgrijalva/jwt-go v3.2.0+incompatible github.com/docker/go-connections v0.4.0 github.com/fatih/color v1.7.0 @@ -16,12 +16,15 @@ require ( github.com/mattn/go-colorable v0.1.2 // indirect github.com/mattn/go-runewidth v0.0.4 // indirect github.com/olekukonko/tablewriter v0.0.1 + github.com/onsi/ginkgo v1.11.0 + github.com/onsi/gomega v1.7.0 github.com/pkg/errors v0.8.1 github.com/spf13/cobra v0.0.5 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.4.0 github.com/testcontainers/testcontainers-go v0.0.10 - golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456 // indirect - golang.org/x/tools v0.0.0-20200116203608-1c4842a210a7 // indirect - gopkg.in/yaml.v2 v2.2.4 + golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550 // indirect + golang.org/x/oauth2 v0.0.0-20190402181905-9f3314589c9a + golang.org/x/sync v0.0.0-20190423024810-112230192c58 // indirect + gopkg.in/yaml.v2 v2.2.8 ) diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index f62cba857e..cb252d815b 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -55,6 +55,13 @@ func GetAuthProvider(config *common.Config) (*Provider, error) { provider, err = NewAuthenticationToken(config.Token, defaultTransport) case len(config.TokenFile) > 0: provider, err = NewAuthenticationTokenFromFile(config.TokenFile, defaultTransport) + case len(config.IssuerEndpoint) > 0 || len(config.KeyFile) > 0: + provider, err = NewAuthenticationOAuth2( + config.IssuerEndpoint, + config.ClientID, + config.Audience, + config.KeyFile, + defaultTransport) } } return &provider, err diff --git a/pulsaradmin/pkg/auth/oauth2.go b/pulsaradmin/pkg/auth/oauth2.go new file mode 100644 index 0000000000..1896b287d2 --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2.go @@ -0,0 +1,94 @@ +// 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 auth + +import ( + "errors" + "fmt" + "net/http" + + "github.com/streamnative/pulsar-admin-go/pkg/auth/oauth2" +) + +type OAuth2Provider struct { + issuer *oauth2.Issuer + keyFile string + T http.RoundTripper +} + +func NewAuthenticationOAuth2( + issueEndpoint, + clientID, + audience, + keyFile string, + transport http.RoundTripper) (*OAuth2Provider, error) { + + return &OAuth2Provider{ + issuer: &oauth2.Issuer{ + IssuerEndpoint: issueEndpoint, + ClientID: clientID, + Audience: audience, + }, + keyFile: keyFile, + T: transport, + }, nil +} + +func (o *OAuth2Provider) RoundTrip(req *http.Request) (*http.Response, error) { + token, err := o.getToken(o.issuer) + if err != nil { + return nil, err + } + req.Header.Add("Authorization", fmt.Sprintf("Bearer %s", token)) + return o.T.RoundTrip(req) +} + +func (o *OAuth2Provider) Transport() http.RoundTripper { + return o.T +} + +func (o *OAuth2Provider) getFlow(issuer *oauth2.Issuer) (oauth2.Flow, error) { + // note that these flows don't rely on the user's cache or configuration + // to produce an ephemeral token that doesn't replace what is generated + // by `login` or by `activate-service-account`. + + var err error + var flow oauth2.Flow + if o.keyFile != "" { + flow, err = oauth2.NewDefaultClientCredentialsFlow(*issuer, o.keyFile) + if err != nil { + return nil, err + } + return flow, err + } + return flow, errors.New("the key file must be specified") +} + +func (o *OAuth2Provider) getToken(issuer *oauth2.Issuer) (string, error) { + flow, err := o.getFlow(issuer) + if err != nil { + return "", err + } + + _, token, err := flow.Authorize() + if err != nil { + return "", err + } + + return token.AccessToken, nil +} diff --git a/pulsaradmin/pkg/auth/oauth2/auth.go b/pulsaradmin/pkg/auth/oauth2/auth.go new file mode 100644 index 0000000000..d6d99f04b5 --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/auth.go @@ -0,0 +1,88 @@ +// 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 oauth2 + +import ( + "fmt" + "time" + + "github.com/dgrijalva/jwt-go" + "github.com/streamnative/pulsar-admin-go/pkg/auth/oauth2/plugin" + "golang.org/x/oauth2" +) + +const ( + ClaimNameUserName = "https://pulsar.apache.org/username" +) + +// Flow abstracts an OAuth 2.0 authentication and authorization flow +type Flow interface { + // Authorize obtains an authorization grant based on an OAuth 2.0 authorization flow. + // The method returns a grant and (optionally) an initial access token. + Authorize() (AuthorizationGrant, *oauth2.Token, error) +} + +// AuthorizationGrant is a credential representing the resource owner's authorization +// to access its protected resources, used by the client to obtain an access token +type AuthorizationGrant interface { + // Refresh obtains a fresh access token based on this grant + Refresh() (*oauth2.Token, error) +} + +// TokenResult holds token information +type TokenResult struct { + AccessToken string `json:"access_token"` + IDToken string `json:"id_token"` + RefreshToken string `json:"refresh_token"` + ExpiresIn int `json:"expires_in"` +} + +// Issuer holds information about the issuer of tokens +type Issuer struct { + IssuerEndpoint string + ClientID string + Audience string +} + +func convertToOAuth2Token(token *TokenResult, clock plugin.Clock) oauth2.Token { + return oauth2.Token{ + AccessToken: token.AccessToken, + TokenType: "bearer", + RefreshToken: token.RefreshToken, + Expiry: clock.Now().Add(time.Duration(token.ExpiresIn) * time.Second), + } +} + +// ExtractUserName extracts the username claim from an authorization grant +func ExtractUserName(token oauth2.Token) (string, error) { + p := jwt.Parser{} + claims := jwt.MapClaims{} + if _, _, err := p.ParseUnverified(token.AccessToken, claims); err != nil { + return "", fmt.Errorf("unable to decode the access token: %v", err) + } + username, ok := claims[ClaimNameUserName] + if !ok { + return "", fmt.Errorf("access token doesn't contain a username claim") + } + switch v := username.(type) { + case string: + return v, nil + default: + return "", fmt.Errorf("access token contains an unsupported username claim") + } +} diff --git a/pulsaradmin/pkg/auth/oauth2/auth_suite_test.go b/pulsaradmin/pkg/auth/oauth2/auth_suite_test.go new file mode 100644 index 0000000000..d35282b52e --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/auth_suite_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 oauth2 + +import ( + "context" + "testing" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" +) + +func TestAuth(t *testing.T) { + RegisterFailHandler(Fail) + RunSpecs(t, "Pulsarctl Auth Suite") +} + +type MockTokenExchanger struct { + CalledWithRequest interface{} + ReturnsTokens *TokenResult + ReturnsError error + RefreshCalledWithRequest *RefreshTokenExchangeRequest +} + +func (te *MockTokenExchanger) ExchangeCode(req AuthorizationCodeExchangeRequest) (*TokenResult, error) { + te.CalledWithRequest = &req + return te.ReturnsTokens, te.ReturnsError +} + +func (te *MockTokenExchanger) ExchangeRefreshToken(req RefreshTokenExchangeRequest) (*TokenResult, error) { + te.RefreshCalledWithRequest = &req + return te.ReturnsTokens, te.ReturnsError +} + +func (te *MockTokenExchanger) ExchangeClientCredentials(req ClientCredentialsExchangeRequest) (*TokenResult, error) { + te.CalledWithRequest = &req + return te.ReturnsTokens, te.ReturnsError +} + +func (te *MockTokenExchanger) ExchangeDeviceCode(ctx context.Context, + req DeviceCodeExchangeRequest) (*TokenResult, error) { + te.CalledWithRequest = &req + return te.ReturnsTokens, te.ReturnsError +} diff --git a/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever.go b/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever.go new file mode 100644 index 0000000000..4723482456 --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever.go @@ -0,0 +1,338 @@ +// 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 oauth2 + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "net/http" + "net/url" + "strconv" + "strings" + "time" +) + +// TokenRetriever implements AuthTokenExchanger in order to facilitate getting +// Tokens +type TokenRetriever struct { + oidcWellKnownEndpoints OIDCWellKnownEndpoints + transport HTTPAuthTransport +} + +// AuthorizationTokenResponse is the HTTP response when asking for a new token. +// Note that not all fields will contain data based on what kind of request was +// sent +type AuthorizationTokenResponse struct { + AccessToken string `json:"access_token"` + ExpiresIn int `json:"expires_in"` + IDToken string `json:"id_token"` + RefreshToken string `json:"refresh_token"` + TokenType string `json:"token_type"` +} + +// AuthorizationCodeExchangeRequest is used to request the exchange of an +// authorization code for a token +type AuthorizationCodeExchangeRequest struct { + ClientID string + CodeVerifier string + Code string + RedirectURI string +} + +// RefreshTokenExchangeRequest is used to request the exchange of a refresh +// token for a refreshed token +type RefreshTokenExchangeRequest struct { + ClientID string + RefreshToken string +} + +// ClientCredentialsExchangeRequest is used to request the exchange of +// client credentials for a token +type ClientCredentialsExchangeRequest struct { + ClientID string + ClientSecret string + Audience string +} + +// DeviceCodeExchangeRequest is used to request the exchange of +// a device code for a token +type DeviceCodeExchangeRequest struct { + ClientID string + DeviceCode string + PollInterval time.Duration +} + +// TokenErrorResponse is used to parse error responses from the token endpoint +type TokenErrorResponse struct { + Error string `json:"error"` + ErrorDescription string `json:"error_description"` +} + +type TokenError struct { + ErrorCode string + ErrorDescription string +} + +func (e *TokenError) Error() string { + if e.ErrorDescription != "" { + return fmt.Sprintf("%s (%s)", e.ErrorDescription, e.ErrorCode) + } + return e.ErrorCode +} + +// HTTPAuthTransport abstracts how an HTTP exchange request is sent and received +type HTTPAuthTransport interface { + Do(request *http.Request) (*http.Response, error) +} + +// NewTokenRetriever allows a TokenRetriever the internal of a new +// TokenRetriever to be easily set up +func NewTokenRetriever( + oidcWellKnownEndpoints OIDCWellKnownEndpoints, + authTransport HTTPAuthTransport) *TokenRetriever { + return &TokenRetriever{ + oidcWellKnownEndpoints: oidcWellKnownEndpoints, + transport: authTransport, + } +} + +// newExchangeCodeRequest builds a new AuthTokenRequest wrapped in an +// http.Request +func (ce *TokenRetriever) newExchangeCodeRequest( + req AuthorizationCodeExchangeRequest) (*http.Request, error) { + uv := url.Values{} + uv.Set("grant_type", "authorization_code") + uv.Set("client_id", req.ClientID) + uv.Set("code_verifier", req.CodeVerifier) + uv.Set("code", req.Code) + uv.Set("redirect_uri", req.RedirectURI) + + euv := uv.Encode() + + request, err := http.NewRequest("POST", + ce.oidcWellKnownEndpoints.TokenEndpoint, + strings.NewReader(euv), + ) + if err != nil { + return nil, err + } + + request.Header.Add("Content-Type", "application/x-www-form-urlencoded") + request.Header.Add("Content-Length", strconv.Itoa(len(euv))) + + return request, nil +} + +// newDeviceCodeExchangeRequest builds a new DeviceCodeExchangeRequest wrapped in an +// http.Request +func (ce *TokenRetriever) newDeviceCodeExchangeRequest( + req DeviceCodeExchangeRequest) (*http.Request, error) { + uv := url.Values{} + uv.Set("grant_type", "urn:ietf:params:oauth:grant-type:device_code") + uv.Set("client_id", req.ClientID) + uv.Set("device_code", req.DeviceCode) + euv := uv.Encode() + + request, err := http.NewRequest("POST", + ce.oidcWellKnownEndpoints.TokenEndpoint, + strings.NewReader(euv), + ) + if err != nil { + return nil, err + } + + request.Header.Add("Content-Type", "application/x-www-form-urlencoded") + request.Header.Add("Content-Length", strconv.Itoa(len(euv))) + + return request, nil +} + +// newRefreshTokenRequest builds a new RefreshTokenRequest wrapped in an +// http.Request +func (ce *TokenRetriever) newRefreshTokenRequest(req RefreshTokenExchangeRequest) (*http.Request, error) { + uv := url.Values{} + uv.Set("grant_type", "refresh_token") + uv.Set("client_id", req.ClientID) + uv.Set("refresh_token", req.RefreshToken) + + euv := uv.Encode() + + request, err := http.NewRequest("POST", + ce.oidcWellKnownEndpoints.TokenEndpoint, + strings.NewReader(euv), + ) + if err != nil { + return nil, err + } + + request.Header.Add("Content-Type", "application/x-www-form-urlencoded") + request.Header.Add("Content-Length", strconv.Itoa(len(euv))) + + return request, nil +} + +// newClientCredentialsRequest builds a new ClientCredentialsExchangeRequest wrapped in an +// http.Request +func (ce *TokenRetriever) newClientCredentialsRequest(req ClientCredentialsExchangeRequest) (*http.Request, error) { + uv := url.Values{} + uv.Set("grant_type", "client_credentials") + uv.Set("client_id", req.ClientID) + uv.Set("client_secret", req.ClientSecret) + uv.Set("audience", req.Audience) + + euv := uv.Encode() + + request, err := http.NewRequest("POST", + ce.oidcWellKnownEndpoints.TokenEndpoint, + strings.NewReader(euv), + ) + if err != nil { + return nil, err + } + + request.Header.Add("Content-Type", "application/x-www-form-urlencoded") + request.Header.Add("Content-Length", strconv.Itoa(len(euv))) + + return request, nil +} + +// ExchangeCode uses the AuthCodeExchangeRequest to exchange an authorization +// code for tokens +func (ce *TokenRetriever) ExchangeCode(req AuthorizationCodeExchangeRequest) (*TokenResult, error) { + request, err := ce.newExchangeCodeRequest(req) + if err != nil { + return nil, err + } + + response, err := ce.transport.Do(request) + if err != nil { + return nil, err + } + + return ce.handleAuthTokensResponse(response) +} + +// handleAuthTokensResponse takes care of checking an http.Response that has +// auth tokens for errors and parsing the raw body to a TokenResult struct +func (ce *TokenRetriever) handleAuthTokensResponse(resp *http.Response) (*TokenResult, error) { + if resp.Body != nil { + defer resp.Body.Close() + } + + if resp.StatusCode < 200 || resp.StatusCode > 299 { + if resp.Header.Get("Content-Type") == "application/json" { + er := TokenErrorResponse{} + err := json.NewDecoder(resp.Body).Decode(&er) + if err != nil { + return nil, err + } + return nil, &TokenError{ErrorCode: er.Error, ErrorDescription: er.ErrorDescription} + } + return nil, fmt.Errorf("a non-success status code was received: %d", resp.StatusCode) + } + + atr := AuthorizationTokenResponse{} + err := json.NewDecoder(resp.Body).Decode(&atr) + if err != nil { + return nil, err + } + + return &TokenResult{ + AccessToken: atr.AccessToken, + IDToken: atr.IDToken, + RefreshToken: atr.RefreshToken, + ExpiresIn: atr.ExpiresIn, + }, nil +} + +// ExchangeDeviceCode uses the DeviceCodeExchangeRequest to exchange a device +// code for tokens +func (ce *TokenRetriever) ExchangeDeviceCode(ctx context.Context, req DeviceCodeExchangeRequest) (*TokenResult, error) { + for { + request, err := ce.newDeviceCodeExchangeRequest(req) + if err != nil { + return nil, err + } + + response, err := ce.transport.Do(request) + if err != nil { + return nil, err + } + token, err := ce.handleAuthTokensResponse(response) + if err == nil { + return token, nil + } + terr, ok := err.(*TokenError) + if !ok { + return nil, err + } + switch terr.ErrorCode { + case "expired_token": + // The user has not authorized the device quickly enough, so the device_code has expired. + return nil, fmt.Errorf("the device code has expired") + case "access_denied": + // The user refused to authorize the device + return nil, fmt.Errorf("the device was not authorized") + case "authorization_pending": + // Still waiting for the user to take action + case "slow_down": + // You are polling too fast + } + + select { + case <-time.After(req.PollInterval): + continue + case <-ctx.Done(): + return nil, errors.New("cancelled") + } + } +} + +// ExchangeRefreshToken uses the RefreshTokenExchangeRequest to exchange a +// refresh token for refreshed tokens +func (ce *TokenRetriever) ExchangeRefreshToken(req RefreshTokenExchangeRequest) (*TokenResult, error) { + request, err := ce.newRefreshTokenRequest(req) + if err != nil { + return nil, err + } + + response, err := ce.transport.Do(request) + if err != nil { + return nil, err + } + + return ce.handleAuthTokensResponse(response) +} + +// ExchangeClientCredentials uses the ClientCredentialsExchangeRequest to exchange +// client credentials for tokens +func (ce *TokenRetriever) ExchangeClientCredentials(req ClientCredentialsExchangeRequest) (*TokenResult, error) { + request, err := ce.newClientCredentialsRequest(req) + if err != nil { + return nil, err + } + + response, err := ce.transport.Do(request) + if err != nil { + return nil, err + } + + return ce.handleAuthTokensResponse(response) +} diff --git a/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever_test.go b/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever_test.go new file mode 100644 index 0000000000..affe255244 --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever_test.go @@ -0,0 +1,334 @@ +// 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 oauth2 + +import ( + "bytes" + "context" + "encoding/json" + "io/ioutil" + "net/http" + "strings" + "time" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" +) + +type MockTransport struct { + Responses []*http.Response + ReturnError error +} + +var _ HTTPAuthTransport = &MockTransport{} + +func (t *MockTransport) Do(req *http.Request) (*http.Response, error) { + if len(t.Responses) > 0 { + r := t.Responses[0] + t.Responses = t.Responses[1:] + return r, nil + } + return nil, t.ReturnError +} + +var _ = Describe("CodetokenExchanger", func() { + Describe("newExchangeCodeRequest", func() { + It("creates the request", func() { + tokenRetriever := TokenRetriever{ + oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "https://issuer/oauth/token"}} + exchangeRequest := AuthorizationCodeExchangeRequest{ + ClientID: "clientID", + CodeVerifier: "Verifier", + Code: "code", + RedirectURI: "https://redirect", + } + + result, err := tokenRetriever.newExchangeCodeRequest(exchangeRequest) + + result.ParseForm() + + Expect(err).To(BeNil()) + Expect(result.FormValue("grant_type")).To(Equal("authorization_code")) + Expect(result.FormValue("client_id")).To(Equal("clientID")) + Expect(result.FormValue("code_verifier")).To(Equal("Verifier")) + Expect(result.FormValue("code")).To(Equal("code")) + Expect(result.FormValue("redirect_uri")).To(Equal("https://redirect")) + Expect(result.URL.String()).To(Equal("https://issuer/oauth/token")) + + Expect(result.Header.Get("Content-Type")).To(Equal("application/x-www-form-urlencoded")) + Expect(result.Header.Get("Content-Length")).To(Equal("117")) + }) + + It("returns an error when NewRequest returns an error", func() { + tokenRetriever := TokenRetriever{ + oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "://issuer/oauth/token"}} + + result, err := tokenRetriever.newExchangeCodeRequest(AuthorizationCodeExchangeRequest{}) + + Expect(result).To(BeNil()) + Expect(err.Error()).To(Equal("parse ://issuer/oauth/token: missing protocol scheme")) + }) + }) + + Describe("handleAuthTokensResponse", func() { + It("handles the response", func() { + tokenRetriever := TokenRetriever{} + response := buildResponse(200, AuthorizationTokenResponse{ + ExpiresIn: 1, + AccessToken: "myAccessToken", + RefreshToken: "myRefreshToken", + }) + + result, err := tokenRetriever.handleAuthTokensResponse(response) + + Expect(err).To(BeNil()) + Expect(result).To(Equal(&TokenResult{ + ExpiresIn: 1, + AccessToken: "myAccessToken", + RefreshToken: "myRefreshToken", + })) + }) + + It("returns error when status code is not successful", func() { + tokenRetriever := TokenRetriever{} + response := buildResponse(500, nil) + + result, err := tokenRetriever.handleAuthTokensResponse(response) + + Expect(result).To(BeNil()) + Expect(err.Error()).To(Equal("a non-success status code was received: 500")) + }) + + It("returns typed error when response body contains error information", func() { + errorBody := TokenErrorResponse{Error: "test", ErrorDescription: "test description"} + tokenRetriever := TokenRetriever{} + response := buildResponse(400, errorBody) + + result, err := tokenRetriever.handleAuthTokensResponse(response) + + Expect(result).To(BeNil()) + Expect(err).To(Equal(&TokenError{ErrorCode: "test", ErrorDescription: "test description"})) + Expect(err.Error()).To(Equal("test description (test)")) + }) + + It("returns error when deserialization fails", func() { + tokenRetriever := TokenRetriever{} + response := buildResponse(200, "") + + result, err := tokenRetriever.handleAuthTokensResponse(response) + Expect(result).To(BeNil()) + Expect(err.Error()).To(Equal( + "json: cannot unmarshal string into Go value of type oauth2.AuthorizationTokenResponse")) + }) + }) + + Describe("newRefreshTokenRequest", func() { + It("creates the request", func() { + tokenRetriever := TokenRetriever{ + oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "https://issuer/oauth/token"}} + exchangeRequest := RefreshTokenExchangeRequest{ + ClientID: "clientID", + RefreshToken: "refreshToken", + } + + result, err := tokenRetriever.newRefreshTokenRequest(exchangeRequest) + + result.ParseForm() + + Expect(err).To(BeNil()) + Expect(result.FormValue("grant_type")).To(Equal("refresh_token")) + Expect(result.FormValue("client_id")).To(Equal("clientID")) + Expect(result.FormValue("refresh_token")).To(Equal("refreshToken")) + Expect(result.URL.String()).To(Equal("https://issuer/oauth/token")) + + Expect(result.Header.Get("Content-Type")).To(Equal("application/x-www-form-urlencoded")) + Expect(result.Header.Get("Content-Length")).To(Equal("70")) + }) + + It("returns an error when NewRequest returns an error", func() { + tokenRetriever := TokenRetriever{ + oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "://issuer/oauth/token"}} + + result, err := tokenRetriever.newRefreshTokenRequest(RefreshTokenExchangeRequest{}) + + Expect(result).To(BeNil()) + Expect(err.Error()).To(Equal("parse ://issuer/oauth/token: missing protocol scheme")) + }) + }) + + Describe("newClientCredentialsRequest", func() { + It("creates the request", func() { + tokenRetriever := TokenRetriever{ + oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "https://issuer/oauth/token"}} + exchangeRequest := ClientCredentialsExchangeRequest{ + ClientID: "clientID", + ClientSecret: "clientSecret", + Audience: "audience", + } + + result, err := tokenRetriever.newClientCredentialsRequest(exchangeRequest) + + result.ParseForm() + + Expect(err).To(BeNil()) + Expect(result.FormValue("grant_type")).To(Equal("client_credentials")) + Expect(result.FormValue("client_id")).To(Equal("clientID")) + Expect(result.FormValue("client_secret")).To(Equal("clientSecret")) + Expect(result.FormValue("audience")).To(Equal("audience")) + Expect(result.URL.String()).To(Equal("https://issuer/oauth/token")) + + Expect(result.Header.Get("Content-Type")).To(Equal("application/x-www-form-urlencoded")) + Expect(result.Header.Get("Content-Length")).To(Equal("93")) + }) + + It("returns an error when NewRequest returns an error", func() { + tokenRetriever := TokenRetriever{ + oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "://issuer/oauth/token"}} + + result, err := tokenRetriever.newClientCredentialsRequest(ClientCredentialsExchangeRequest{}) + + Expect(result).To(BeNil()) + Expect(err.Error()).To(Equal("parse ://issuer/oauth/token: missing protocol scheme")) + }) + }) + + Describe("newDeviceCodeExchangeRequest", func() { + It("creates the request", func() { + tokenRetriever := TokenRetriever{ + oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "https://issuer/oauth/token"}} + exchangeRequest := DeviceCodeExchangeRequest{ + ClientID: "clientID", + DeviceCode: "deviceCode", + PollInterval: time.Duration(5) * time.Second, + } + + result, err := tokenRetriever.newDeviceCodeExchangeRequest(exchangeRequest) + + result.ParseForm() + + Expect(err).To(BeNil()) + Expect(result.FormValue("grant_type")).To(Equal("urn:ietf:params:oauth:grant-type:device_code")) + Expect(result.FormValue("client_id")).To(Equal("clientID")) + Expect(result.FormValue("device_code")).To(Equal("deviceCode")) + Expect(result.URL.String()).To(Equal("https://issuer/oauth/token")) + + Expect(result.Header.Get("Content-Type")).To(Equal("application/x-www-form-urlencoded")) + Expect(result.Header.Get("Content-Length")).To(Equal("107")) + }) + + It("returns an error when NewRequest returns an error", func() { + tokenRetriever := TokenRetriever{ + oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "://issuer/oauth/token"}} + + result, err := tokenRetriever.newClientCredentialsRequest(ClientCredentialsExchangeRequest{}) + + Expect(result).To(BeNil()) + Expect(err.Error()).To(Equal("parse ://issuer/oauth/token: missing protocol scheme")) + }) + }) + + Describe("ExchangeDeviceCode", func() { + var mockTransport *MockTransport + var tokenRetriever *TokenRetriever + var exchangeRequest DeviceCodeExchangeRequest + var tokenResult TokenResult + + BeforeEach(func() { + mockTransport = &MockTransport{} + tokenRetriever = &TokenRetriever{ + oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "https://issuer/oauth/token"}, + transport: mockTransport, + } + exchangeRequest = DeviceCodeExchangeRequest{ + ClientID: "clientID", + DeviceCode: "deviceCode", + PollInterval: time.Duration(1) * time.Second, + } + tokenResult = TokenResult{ + ExpiresIn: 1, + AccessToken: "myAccessToken", + RefreshToken: "myRefreshToken", + } + }) + + It("returns a token", func() { + }) + + It("supports cancellation", func() { + mockTransport.Responses = []*http.Response{ + buildResponse(400, &TokenErrorResponse{"authorization_pending", ""}), + } + ctx, cancel := context.WithCancel(context.Background()) + cancel() + _, err := tokenRetriever.ExchangeDeviceCode(ctx, exchangeRequest) + Expect(err).ToNot(BeNil()) + Expect(err.Error()).To(Equal("cancelled")) + }) + + It("implements authorization_pending and slow_down", func() { + startTime := time.Now() + mockTransport.Responses = []*http.Response{ + buildResponse(400, &TokenErrorResponse{"authorization_pending", ""}), + buildResponse(400, &TokenErrorResponse{"authorization_pending", ""}), + buildResponse(400, &TokenErrorResponse{"slow_down", ""}), + buildResponse(200, &tokenResult), + } + token, err := tokenRetriever.ExchangeDeviceCode(context.Background(), exchangeRequest) + Expect(err).To(BeNil()) + Expect(token).To(Equal(&tokenResult)) + endTime := time.Now() + Expect(endTime.Sub(startTime)).To(BeNumerically(">", exchangeRequest.PollInterval*3)) + }) + + It("implements expired_token", func() { + mockTransport.Responses = []*http.Response{ + buildResponse(400, &TokenErrorResponse{"expired_token", ""}), + } + _, err := tokenRetriever.ExchangeDeviceCode(context.Background(), exchangeRequest) + Expect(err).ToNot(BeNil()) + Expect(err.Error()).To(Equal("the device code has expired")) + }) + + It("implements access_denied", func() { + mockTransport.Responses = []*http.Response{ + buildResponse(400, &TokenErrorResponse{"access_denied", ""}), + } + _, err := tokenRetriever.ExchangeDeviceCode(context.Background(), exchangeRequest) + Expect(err).ToNot(BeNil()) + Expect(err.Error()).To(Equal("the device was not authorized")) + }) + }) +}) + +func buildResponse(statusCode int, body interface{}) *http.Response { + b, err := json.Marshal(body) + if err != nil { + panic(err) + } + + resp := &http.Response{ + StatusCode: statusCode, + Header: map[string][]string{}, + Body: ioutil.NopCloser(bytes.NewReader(b)), + } + if strings.HasPrefix(string(b), "{") { + resp.Header.Add("Content-Type", "application/json") + } + + return resp +} diff --git a/pulsaradmin/pkg/auth/oauth2/client_credentials_flow.go b/pulsaradmin/pkg/auth/oauth2/client_credentials_flow.go new file mode 100644 index 0000000000..ba1a187ce0 --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/client_credentials_flow.go @@ -0,0 +1,149 @@ +// 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 oauth2 + +import ( + "net/http" + + "github.com/streamnative/pulsar-admin-go/pkg/auth/oauth2/plugin" + "golang.org/x/oauth2" + + "github.com/pkg/errors" +) + +// ClientCredentialsFlow takes care of the mechanics needed for getting an access +// token using the OAuth 2.0 "Client Credentials Flow" +type ClientCredentialsFlow struct { + issuerData Issuer + provider ClientCredentialsProvider + exchanger ClientCredentialsExchanger + clock plugin.Clock +} + +// ClientCredentialsProvider abstracts getting client credentials +type ClientCredentialsProvider interface { + GetClientCredentials() (*KeyFile, error) +} + +// ClientCredentialsExchanger abstracts exchanging client credentials for tokens +type ClientCredentialsExchanger interface { + ExchangeClientCredentials(req ClientCredentialsExchangeRequest) (*TokenResult, error) +} + +func NewClientCredentialsFlow( + issuerData Issuer, + provider ClientCredentialsProvider, + exchanger ClientCredentialsExchanger, + clock plugin.Clock) *ClientCredentialsFlow { + return &ClientCredentialsFlow{ + issuerData: issuerData, + provider: provider, + exchanger: exchanger, + clock: clock, + } +} + +// NewDefaultClientCredentialsFlow provides an easy way to build up a default +// client credentials flow with all the correct configuration. +func NewDefaultClientCredentialsFlow(issuerData Issuer, keyFile string) (*ClientCredentialsFlow, error) { + wellKnownEndpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL(issuerData.IssuerEndpoint) + if err != nil { + return nil, err + } + + credsProvider := NewClientCredentialsProviderFromKeyFile(keyFile) + + tokenRetriever := NewTokenRetriever( + *wellKnownEndpoints, + &http.Client{}) + + return NewClientCredentialsFlow( + issuerData, + credsProvider, + tokenRetriever, + plugin.RealClock{}), nil +} + +var _ Flow = &ClientCredentialsFlow{} + +func (c *ClientCredentialsFlow) Authorize() (AuthorizationGrant, *oauth2.Token, error) { + keyFile, err := c.provider.GetClientCredentials() + if err != nil { + return nil, nil, errors.Wrap(err, "could not get client credentials") + } + + grant := &ClientCredentialsGrant{ + KeyFile: *keyFile, + issuerData: c.issuerData, + exchanger: c.exchanger, + clock: c.clock, + } + + // test the credentials and obtain an initial access token + token, err := grant.Refresh() + if err != nil { + return nil, nil, errors.Wrap(err, "authentication failed using client credentials") + } + + return grant, token, nil +} + +type ClientCredentialsGrant struct { + KeyFile KeyFile + issuerData Issuer + exchanger ClientCredentialsExchanger + clock plugin.Clock +} + +func NewDefaultClientCredentialsGrant(issuerData Issuer, keyFile KeyFile, + clock plugin.Clock) (*ClientCredentialsGrant, error) { + wellKnownEndpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL(issuerData.IssuerEndpoint) + if err != nil { + return nil, err + } + + tokenRetriever := NewTokenRetriever( + *wellKnownEndpoints, + &http.Client{}) + + return &ClientCredentialsGrant{ + KeyFile: keyFile, + issuerData: issuerData, + exchanger: tokenRetriever, + clock: clock, + }, nil +} + +var _ AuthorizationGrant = &ClientCredentialsGrant{} + +func (g *ClientCredentialsGrant) Refresh() (*oauth2.Token, error) { + exchangeRequest := ClientCredentialsExchangeRequest{ + Audience: g.issuerData.Audience, + ClientID: g.KeyFile.ClientID, + ClientSecret: g.KeyFile.ClientSecret, + } + + tr, err := g.exchanger.ExchangeClientCredentials(exchangeRequest) + if err != nil { + return nil, errors.Wrap(err, "could not exchange client credentials") + } + + token := convertToOAuth2Token(tr, g.clock) + + return &token, nil +} diff --git a/pulsaradmin/pkg/auth/oauth2/client_credentials_flow_test.go b/pulsaradmin/pkg/auth/oauth2/client_credentials_flow_test.go new file mode 100644 index 0000000000..6ac4be4064 --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/client_credentials_flow_test.go @@ -0,0 +1,135 @@ +// 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 oauth2 + +import ( + "errors" + "time" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" + "github.com/streamnative/pulsar-admin-go/pkg/auth/oauth2/plugin" +) + +type MockClientCredentialsProvider struct { + Called bool + ClientCredentialsResult *KeyFile + ReturnsError error +} + +func (m *MockClientCredentialsProvider) GetClientCredentials() (*KeyFile, error) { + m.Called = true + return m.ClientCredentialsResult, m.ReturnsError +} + +var _ ClientCredentialsProvider = &MockClientCredentialsProvider{} + +var _ = Describe("ClientCredentialsFlow", func() { + issuer := Issuer{ + IssuerEndpoint: "http://issuer", + ClientID: "", + Audience: "test_audience", + } + + Describe("Authorize", func() { + + var mockClock plugin.Clock + var mockCredsProvider *MockClientCredentialsProvider + var mockTokenExchanger *MockTokenExchanger + + BeforeEach(func() { + mockClock = plugin.NewFakeClock(time.Unix(0, 0)) + + mockCredsProvider = &MockClientCredentialsProvider{ + ClientCredentialsResult: &KeyFile{ + Type: KeyFileTypeServiceAccount, + ClientID: "test_clientID", + ClientSecret: "test_clientSecret", + ClientEmail: "test_clientEmail", + }, + } + + expectedTokens := TokenResult{AccessToken: "accessToken", RefreshToken: "refreshToken", ExpiresIn: 1234} + mockTokenExchanger = &MockTokenExchanger{ + ReturnsTokens: &expectedTokens, + } + }) + + It("invokes TokenExchanger with credentials", func() { + provider := NewClientCredentialsFlow( + issuer, + mockCredsProvider, + mockTokenExchanger, + mockClock, + ) + + _, _, err := provider.Authorize() + Expect(err).ToNot(HaveOccurred()) + Expect(mockCredsProvider.Called).To(BeTrue()) + Expect(mockTokenExchanger.CalledWithRequest).To(Equal(&ClientCredentialsExchangeRequest{ + ClientID: mockCredsProvider.ClientCredentialsResult.ClientID, + ClientSecret: mockCredsProvider.ClientCredentialsResult.ClientSecret, + Audience: issuer.Audience, + })) + }) + + It("returns TokensResult from TokenExchanger", func() { + provider := NewClientCredentialsFlow( + issuer, + mockCredsProvider, + mockTokenExchanger, + mockClock, + ) + + _, token, err := provider.Authorize() + Expect(err).ToNot(HaveOccurred()) + expected := convertToOAuth2Token(mockTokenExchanger.ReturnsTokens, mockClock) + Expect(*token).To(Equal(expected)) + }) + + It("returns an error if client credentials request errors", func() { + mockCredsProvider.ReturnsError = errors.New("someerror") + + provider := NewClientCredentialsFlow( + issuer, + mockCredsProvider, + mockTokenExchanger, + mockClock, + ) + + _, _, err := provider.Authorize() + Expect(err.Error()).To(Equal("could not get client credentials: someerror")) + }) + + It("returns an error if token exchanger errors", func() { + mockTokenExchanger.ReturnsError = errors.New("someerror") + mockTokenExchanger.ReturnsTokens = nil + + provider := NewClientCredentialsFlow( + issuer, + mockCredsProvider, + mockTokenExchanger, + mockClock, + ) + + _, _, err := provider.Authorize() + Expect(err.Error()).To(Equal("authentication failed using client credentials: " + + "could not exchange client credentials: someerror")) + }) + }) +}) diff --git a/pulsaradmin/pkg/auth/oauth2/client_credentials_provider.go b/pulsaradmin/pkg/auth/oauth2/client_credentials_provider.go new file mode 100644 index 0000000000..48b2e46d39 --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/client_credentials_provider.go @@ -0,0 +1,65 @@ +// 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 oauth2 + +import ( + "encoding/json" + "fmt" + "io/ioutil" +) + +const ( + KeyFileTypeServiceAccount = "sn_service_account" +) + +type KeyFileProvider struct { + KeyFile string +} + +type KeyFile struct { + Type string `json:"type"` + ClientID string `json:"client_id"` + ClientSecret string `json:"client_secret"` + ClientEmail string `json:"client_email"` +} + +func NewClientCredentialsProviderFromKeyFile(keyFile string) *KeyFileProvider { + return &KeyFileProvider{ + KeyFile: keyFile, + } +} + +var _ ClientCredentialsProvider = &KeyFileProvider{} + +func (k *KeyFileProvider) GetClientCredentials() (*KeyFile, error) { + keyFile, err := ioutil.ReadFile(k.KeyFile) + if err != nil { + return nil, err + } + + var v KeyFile + err = json.Unmarshal(keyFile, &v) + if err != nil { + return nil, err + } + if v.Type != KeyFileTypeServiceAccount { + return nil, fmt.Errorf("open %s: unsupported format", k.KeyFile) + } + + return &v, nil +} diff --git a/pulsaradmin/pkg/auth/oauth2/config_tokenprovider.go b/pulsaradmin/pkg/auth/oauth2/config_tokenprovider.go new file mode 100644 index 0000000000..627749fb70 --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/config_tokenprovider.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 oauth2 + +import "fmt" + +type configProvider interface { + GetTokens(identifier string) (string, string) + SaveTokens(identifier, accessToken, refreshToken string) +} + +// ConfigBackedCachingProvider wraps a configProvider in order to conform to +// the cachingProvider interface +type ConfigBackedCachingProvider struct { + identifier string + config configProvider +} + +// NewConfigBackedCachingProvider builds and returns a CachingTokenProvider +// that utilizes a configProvider to cache tokens +func NewConfigBackedCachingProvider(clientID, audience string, config configProvider) *ConfigBackedCachingProvider { + return &ConfigBackedCachingProvider{ + identifier: fmt.Sprintf("%s-%s", clientID, audience), + config: config, + } +} + +// GetTokens gets the tokens from the cache and returns them as a TokenResult +func (c *ConfigBackedCachingProvider) GetTokens() (*TokenResult, error) { + accessToken, refreshToken := c.config.GetTokens(c.identifier) + return &TokenResult{ + AccessToken: accessToken, + RefreshToken: refreshToken, + }, nil +} + +// CacheTokens caches the id and refresh token from TokenResult in the +// configProvider +func (c *ConfigBackedCachingProvider) CacheTokens(toCache *TokenResult) error { + c.config.SaveTokens(c.identifier, toCache.AccessToken, toCache.RefreshToken) + return nil +} diff --git a/pulsaradmin/pkg/auth/oauth2/config_tokenprovider_test.go b/pulsaradmin/pkg/auth/oauth2/config_tokenprovider_test.go new file mode 100644 index 0000000000..d949a5a5e9 --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/config_tokenprovider_test.go @@ -0,0 +1,91 @@ +// 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 oauth2 + +import ( + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" +) + +type mockConfigProvider struct { + ReturnAccessToken string + ReturnRefreshToken string + GetTokensCalledIdentifier string + SavedIdentifier string + SavedAccessToken string + SavedRefreshToken string +} + +func (m *mockConfigProvider) GetTokens(identifier string) (string, string) { + m.GetTokensCalledIdentifier = identifier + return m.ReturnAccessToken, m.ReturnRefreshToken +} + +func (m *mockConfigProvider) SaveTokens(identifier, accessToken, refreshToken string) { + m.SavedIdentifier = identifier + m.SavedAccessToken = accessToken + m.SavedRefreshToken = refreshToken +} + +var _ = Describe("main", func() { + Describe("configCachingProvider", func() { + It("sets up the identifier using the clientID and audience", func() { + p := NewConfigBackedCachingProvider("iamclientid", "iamaudience", &mockConfigProvider{}) + + Expect(p.identifier).To(Equal("iamclientid-iamaudience")) + }) + + It("gets tokens from the config provider", func() { + c := &mockConfigProvider{ + ReturnAccessToken: "accessToken", + ReturnRefreshToken: "refreshToken", + } + p := ConfigBackedCachingProvider{ + identifier: "iamidentifier", + config: c, + } + + r, err := p.GetTokens() + + Expect(err).NotTo(HaveOccurred()) + Expect(c.GetTokensCalledIdentifier).To(Equal(p.identifier)) + Expect(r).To(Equal(&TokenResult{ + AccessToken: c.ReturnAccessToken, + RefreshToken: c.ReturnRefreshToken, + })) + }) + + It("caches the tokens in the config provider", func() { + c := &mockConfigProvider{} + p := ConfigBackedCachingProvider{ + identifier: "iamidentifier", + config: c, + } + toSave := &TokenResult{ + AccessToken: "accessToken", + RefreshToken: "refreshToken", + } + + p.CacheTokens(toSave) + + Expect(c.SavedIdentifier).To(Equal(p.identifier)) + Expect(c.SavedAccessToken).To(Equal(toSave.AccessToken)) + Expect(c.SavedRefreshToken).To(Equal(toSave.RefreshToken)) + }) + }) +}) diff --git a/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider.go b/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider.go new file mode 100644 index 0000000000..32986b7314 --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider.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 oauth2 + +import ( + "encoding/json" + "net/http" + "net/url" + "path" + + "github.com/pkg/errors" +) + +// OIDCWellKnownEndpoints holds the well known OIDC endpoints +type OIDCWellKnownEndpoints struct { + AuthorizationEndpoint string `json:"authorization_endpoint"` + TokenEndpoint string `json:"token_endpoint"` + DeviceAuthorizationEndpoint string `json:"device_authorization_endpoint"` +} + +// GetOIDCWellKnownEndpointsFromIssuerURL gets the well known endpoints for the +// passed in issuer url +func GetOIDCWellKnownEndpointsFromIssuerURL(issuerURL string) (*OIDCWellKnownEndpoints, error) { + u, err := url.Parse(issuerURL) + if err != nil { + return nil, errors.Wrap(err, "could not parse issuer url to build well known endpoints") + } + u.Path = path.Join(u.Path, ".well-known/openid-configuration") + + r, err := http.Get(u.String()) + if err != nil { + return nil, errors.Wrapf(err, "could not get well known endpoints from url %s", u.String()) + } + defer r.Body.Close() + + var wkEndpoints OIDCWellKnownEndpoints + err = json.NewDecoder(r.Body).Decode(&wkEndpoints) + if err != nil { + return nil, errors.Wrap(err, "could not decode json body when getting well known endpoints") + } + + return &wkEndpoints, nil +} diff --git a/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider_test.go b/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider_test.go new file mode 100644 index 0000000000..4ebce3b6ca --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider_test.go @@ -0,0 +1,92 @@ +// 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 oauth2 + +import ( + "encoding/json" + "net/http" + "net/http/httptest" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" +) + +var _ = Describe("GetOIDCWellKnownEndpointsFromIssuerURL", func() { + It("calls and gets the well known data from the correct endpoint for the issuer", func() { + var req *http.Request + wkEndpointsResp := OIDCWellKnownEndpoints{ + AuthorizationEndpoint: "the-auth-endpoint", TokenEndpoint: "the-token-endpoint"} + responseBytes, err := json.Marshal(wkEndpointsResp) + Expect(err).ToNot(HaveOccurred()) + + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + req = r + + w.WriteHeader(http.StatusOK) + w.Write(responseBytes) + + })) + defer ts.Close() + + endpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL(ts.URL) + + Expect(err).ToNot(HaveOccurred()) + Expect(*endpoints).To(Equal(wkEndpointsResp)) + Expect(req.URL.Path).To(Equal("/.well-known/openid-configuration")) + }) + + It("errors when url.Parse errors", func() { + endpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL("://") + + Expect(err).To(HaveOccurred()) + Expect(err.Error()).To(Equal( + "could not parse issuer url to build well known endpoints: parse ://: missing protocol scheme")) + Expect(endpoints).To(BeNil()) + }) + + It("errors when the get errors", func() { + endpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL("https://") + + Expect(err).To(HaveOccurred()) + Expect(err.Error()).To(Equal( + "could not get well known endpoints from url https://.well-known/openid-configuration: " + + "Get https://.well-known/openid-configuration: dial tcp: lookup .well-known: no such host")) + Expect(endpoints).To(BeNil()) + }) + + It("errors when the json decoder errors", func() { + var req *http.Request + + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + req = r + + w.WriteHeader(http.StatusOK) + w.Write([]byte("<")) + + })) + defer ts.Close() + + endpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL(ts.URL) + + Expect(err).To(HaveOccurred()) + Expect(err.Error()).To(Equal("could not decode json body when getting well" + + " known endpoints: invalid character '<' looking for beginning of value")) + Expect(endpoints).To(BeNil()) + Expect(req.URL.Path).To(Equal("/.well-known/openid-configuration")) + }) +}) diff --git a/pulsaradmin/pkg/auth/oauth2/plugin/clock.go b/pulsaradmin/pkg/auth/oauth2/plugin/clock.go new file mode 100644 index 0000000000..b2cf4a985c --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/plugin/clock.go @@ -0,0 +1,99 @@ +// 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 plugin + +import "time" + +// Clock allows for injecting fake or real clocks into code that +// needs to do arbitrary things based on time. +type Clock interface { + Now() time.Time + Since(time.Time) time.Duration + After(d time.Duration) <-chan time.Time + NewTimer(d time.Duration) Timer + Sleep(d time.Duration) + Tick(d time.Duration) <-chan time.Time +} + +var _ = Clock(RealClock{}) + +// RealClock really calls time.Now() +type RealClock struct{} + +// Now returns the current time. +func (RealClock) Now() time.Time { + return time.Now() +} + +// Since returns time since the specified timestamp. +func (RealClock) Since(ts time.Time) time.Duration { + return time.Since(ts) +} + +// After is the same as time.After(d). +func (RealClock) After(d time.Duration) <-chan time.Time { + return time.After(d) +} + +// NewTimer is the same as time.NewTimer(d) +func (RealClock) NewTimer(d time.Duration) Timer { + return &realTimer{ + timer: time.NewTimer(d), + } +} + +// Tick is the same as time.Tick(d) +// nolint +func (RealClock) Tick(d time.Duration) <-chan time.Time { + return time.Tick(d) +} + +// Sleep is the same as time.Sleep(d) +func (RealClock) Sleep(d time.Duration) { + time.Sleep(d) +} + +// Timer allows for injecting fake or real timers into code that +// needs to do arbitrary things based on time. +type Timer interface { + C() <-chan time.Time + Stop() bool + Reset(d time.Duration) bool +} + +var _ = Timer(&realTimer{}) + +// realTimer is backed by an actual time.Timer. +type realTimer struct { + timer *time.Timer +} + +// C returns the underlying timer's channel. +func (r *realTimer) C() <-chan time.Time { + return r.timer.C +} + +// Stop calls Stop() on the underlying timer. +func (r *realTimer) Stop() bool { + return r.timer.Stop() +} + +// Reset calls Reset() on the underlying timer. +func (r *realTimer) Reset(d time.Duration) bool { + return r.timer.Reset(d) +} diff --git a/pulsaradmin/pkg/auth/oauth2/plugin/fake_clock.go b/pulsaradmin/pkg/auth/oauth2/plugin/fake_clock.go new file mode 100644 index 0000000000..099e0793d2 --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2/plugin/fake_clock.go @@ -0,0 +1,270 @@ +// 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 plugin + +import ( + "sync" + "time" +) + +var ( + _ = Clock(&FakeClock{}) + _ = Clock(&IntervalClock{}) +) + +// FakeClock implements Clock, but returns an arbitrary time. +type FakeClock struct { + lock sync.RWMutex + time time.Time + + // waiters are waiting for the fake time to pass their specified time + waiters []*fakeClockWaiter +} + +type fakeClockWaiter struct { + targetTime time.Time + stepInterval time.Duration + skipIfBlocked bool + destChan chan time.Time + fired bool +} + +// NewFakeClock constructs a fake clock set to the provided time. +func NewFakeClock(t time.Time) *FakeClock { + return &FakeClock{ + time: t, + } +} + +// Now returns f's time. +func (f *FakeClock) Now() time.Time { + f.lock.RLock() + defer f.lock.RUnlock() + return f.time +} + +// Since returns time since the time in f. +func (f *FakeClock) Since(ts time.Time) time.Duration { + f.lock.RLock() + defer f.lock.RUnlock() + return f.time.Sub(ts) +} + +// After is the fake version of time.After(d). +func (f *FakeClock) After(d time.Duration) <-chan time.Time { + f.lock.Lock() + defer f.lock.Unlock() + stopTime := f.time.Add(d) + ch := make(chan time.Time, 1) // Don't block! + f.waiters = append(f.waiters, &fakeClockWaiter{ + targetTime: stopTime, + destChan: ch, + }) + return ch +} + +// NewTimer constructs a fake timer, akin to time.NewTimer(d). +func (f *FakeClock) NewTimer(d time.Duration) Timer { + f.lock.Lock() + defer f.lock.Unlock() + stopTime := f.time.Add(d) + ch := make(chan time.Time, 1) // Don't block! + timer := &fakeTimer{ + fakeClock: f, + waiter: fakeClockWaiter{ + targetTime: stopTime, + destChan: ch, + }, + } + f.waiters = append(f.waiters, &timer.waiter) + return timer +} + +// Tick constructs a fake ticker, akin to time.Tick +func (f *FakeClock) Tick(d time.Duration) <-chan time.Time { + if d <= 0 { + return nil + } + f.lock.Lock() + defer f.lock.Unlock() + tickTime := f.time.Add(d) + ch := make(chan time.Time, 1) // hold one tick + f.waiters = append(f.waiters, &fakeClockWaiter{ + targetTime: tickTime, + stepInterval: d, + skipIfBlocked: true, + destChan: ch, + }) + + return ch +} + +// Step moves the clock by Duration and notifies anyone that's called After, +// Tick, or NewTimer. +func (f *FakeClock) Step(d time.Duration) { + f.lock.Lock() + defer f.lock.Unlock() + f.setTimeLocked(f.time.Add(d)) +} + +// SetTime sets the time. +func (f *FakeClock) SetTime(t time.Time) { + f.lock.Lock() + defer f.lock.Unlock() + f.setTimeLocked(t) +} + +// Actually changes the time and checks any waiters. f must be write-locked. +func (f *FakeClock) setTimeLocked(t time.Time) { + f.time = t + newWaiters := make([]*fakeClockWaiter, 0, len(f.waiters)) + for i := range f.waiters { + w := f.waiters[i] + if !w.targetTime.After(t) { + + if w.skipIfBlocked { + select { + case w.destChan <- t: + w.fired = true + default: + } + } else { + w.destChan <- t + w.fired = true + } + + if w.stepInterval > 0 { + for !w.targetTime.After(t) { + w.targetTime = w.targetTime.Add(w.stepInterval) + } + newWaiters = append(newWaiters, w) + } + + } else { + newWaiters = append(newWaiters, f.waiters[i]) + } + } + f.waiters = newWaiters +} + +// HasWaiters returns true if After has been called on f but not yet satisfied (so you can +// write race-free tests). +func (f *FakeClock) HasWaiters() bool { + f.lock.RLock() + defer f.lock.RUnlock() + return len(f.waiters) > 0 +} + +// Sleep is akin to time.Sleep +func (f *FakeClock) Sleep(d time.Duration) { + f.Step(d) +} + +// IntervalClock implements Clock, but each invocation of Now steps the clock forward the specified duration +type IntervalClock struct { + Time time.Time + Duration time.Duration +} + +// Now returns i's time. +func (i *IntervalClock) Now() time.Time { + i.Time = i.Time.Add(i.Duration) + return i.Time +} + +// Since returns time since the time in i. +func (i *IntervalClock) Since(ts time.Time) time.Duration { + return i.Time.Sub(ts) +} + +// After is unimplemented, will panic. +func (*IntervalClock) After(d time.Duration) <-chan time.Time { + panic("IntervalClock doesn't implement After") +} + +// NewTimer is unimplemented, will panic. +func (*IntervalClock) NewTimer(d time.Duration) Timer { + panic("IntervalClock doesn't implement NewTimer") +} + +// Tick is unimplemented, will panic. +func (*IntervalClock) Tick(d time.Duration) <-chan time.Time { + panic("IntervalClock doesn't implement Tick") +} + +// Sleep is unimplemented, will panic. +func (*IntervalClock) Sleep(d time.Duration) { + panic("IntervalClock doesn't implement Sleep") +} + +var _ = Timer(&fakeTimer{}) + +// fakeTimer implements Timer based on a FakeClock. +type fakeTimer struct { + fakeClock *FakeClock + waiter fakeClockWaiter +} + +// C returns the channel that notifies when this timer has fired. +func (f *fakeTimer) C() <-chan time.Time { + return f.waiter.destChan +} + +// Stop stops the timer and returns true if the timer has not yet fired, or false otherwise. +func (f *fakeTimer) Stop() bool { + f.fakeClock.lock.Lock() + defer f.fakeClock.lock.Unlock() + + newWaiters := make([]*fakeClockWaiter, 0, len(f.fakeClock.waiters)) + for i := range f.fakeClock.waiters { + w := f.fakeClock.waiters[i] + if w != &f.waiter { + newWaiters = append(newWaiters, w) + } + } + + f.fakeClock.waiters = newWaiters + + return !f.waiter.fired +} + +// Reset resets the timer to the fake clock's "now" + d. It returns true if the timer has not yet +// fired, or false otherwise. +func (f *fakeTimer) Reset(d time.Duration) bool { + f.fakeClock.lock.Lock() + defer f.fakeClock.lock.Unlock() + + active := !f.waiter.fired + + f.waiter.fired = false + f.waiter.targetTime = f.fakeClock.time.Add(d) + + var isWaiting bool + for i := range f.fakeClock.waiters { + w := f.fakeClock.waiters[i] + if w == &f.waiter { + isWaiting = true + break + } + } + if !isWaiting { + f.fakeClock.waiters = append(f.fakeClock.waiters, &f.waiter) + } + + return active +} diff --git a/pulsaradmin/pkg/auth/oauth2_test.go b/pulsaradmin/pkg/auth/oauth2_test.go new file mode 100644 index 0000000000..7efb7af81d --- /dev/null +++ b/pulsaradmin/pkg/auth/oauth2_test.go @@ -0,0 +1,106 @@ +// 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 auth + +import ( + "fmt" + "io/ioutil" + "net/http" + "net/http/httptest" + "os" + "testing" + + "github.com/pkg/errors" + "github.com/stretchr/testify/assert" +) + +// mockOAuthServer will mock a oauth service for the tests +func mockOAuthServer() *httptest.Server { + // prepare a port for the mocked server + server := httptest.NewUnstartedServer(http.DefaultServeMux) + + // mock the used REST path for the tests + mockedHandler := http.NewServeMux() + mockedHandler.HandleFunc("/.well-known/openid-configuration", func(writer http.ResponseWriter, request *http.Request) { + s := fmt.Sprintf(`{ + "issuer":"%s", + "authorization_endpoint":"%s/authorize", + "token_endpoint":"%s/oauth/token", + "device_authorization_endpoint":"%s/oauth/device/code" +}`, server.URL, server.URL, server.URL, server.URL) + fmt.Fprintln(writer, s) + }) + mockedHandler.HandleFunc("/oauth/token", func(writer http.ResponseWriter, request *http.Request) { + fmt.Fprintln(writer, "{\n \"access_token\": \"token-content\",\n \"token_type\": \"Bearer\"\n}") + }) + mockedHandler.HandleFunc("/authorize", func(writer http.ResponseWriter, request *http.Request) { + fmt.Fprintln(writer, "true") + }) + + server.Config.Handler = mockedHandler + server.Start() + + return server +} + +// mockKeyFile will mock a temp key file for testing. +func mockKeyFile() (string, error) { + pwd, err := os.Getwd() + if err != nil { + return "", err + } + kf, err := ioutil.TempFile(pwd, "test_oauth2") + if err != nil { + return "", err + } + _, err = kf.WriteString(`{ + "type":"sn_service_account", + "client_id":"client-id", + "client_secret":"client-secret", + "client_email":"oauth@test.org", + "issuer_url":"http://issue-url" +}`) + if err != nil { + return "", err + } + + return kf.Name(), nil +} + +func TestOauth2(t *testing.T) { + server := mockOAuthServer() + defer server.Close() + kf, err := mockKeyFile() + defer os.Remove(kf) + if err != nil { + t.Fatal(errors.Wrap(err, "create mocked key file failed")) + } + + transport := http.DefaultTransport.(*http.Transport) + + auth, err := NewAuthenticationOAuth2(server.URL, "client-id", "audience", kf, transport) + if err != nil { + t.Fatal(err) + } + + token, err := auth.getToken(auth.issuer) + if err != nil { + t.Fatal(err) + } + assert.Equal(t, "token-content", token) +} diff --git a/pulsaradmin/pkg/pulsar/common/config.go b/pulsaradmin/pkg/pulsar/common/config.go index ca02fef554..09a4c0c039 100644 --- a/pulsaradmin/pkg/pulsar/common/config.go +++ b/pulsaradmin/pkg/pulsar/common/config.go @@ -42,4 +42,10 @@ type Config struct { Token string TokenFile string PulsarAPIVersion APIVersion + + // OAuth2 configuration + IssuerEndpoint string + ClientID string + Audience string + KeyFile string } From d1a43f8b363ff10b8f6806b4b6aa824df2750430 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 16 Jul 2020 23:49:13 +0800 Subject: [PATCH 169/348] Improve the oauth2 implementation (streamnative/pulsarctl#237) *Motivation* When using pulsarctl as a library, the oauth2 auth provider should initialize the authenticate flow and save the token to the store. When using pulsarctl as a command-line tool, users should login or active at first to get access token and save it to the store, then other commands can get the token from the store. *Modification* - Add login command - Add activate command - Update the oauth2Porvider code --- pulsaradmin/go.mod | 16 +- pulsaradmin/pkg/auth/auth_provider.go | 6 +- pulsaradmin/pkg/auth/oauth2.go | 189 +++++++--- pulsaradmin/pkg/auth/oauth2/auth.go | 88 ----- .../pkg/auth/oauth2/auth_suite_test.go | 59 --- .../oauth2/authorization_tokenretriever.go | 338 ------------------ .../authorization_tokenretriever_test.go | 334 ----------------- .../auth/oauth2/client_credentials_flow.go | 149 -------- .../oauth2/client_credentials_flow_test.go | 135 ------- .../oauth2/client_credentials_provider.go | 65 ---- .../pkg/auth/oauth2/config_tokenprovider.go | 57 --- .../auth/oauth2/config_tokenprovider_test.go | 91 ----- .../pkg/auth/oauth2/oidc_endpoint_provider.go | 58 --- .../oauth2/oidc_endpoint_provider_test.go | 92 ----- pulsaradmin/pkg/auth/oauth2/plugin/clock.go | 99 ----- .../pkg/auth/oauth2/plugin/fake_clock.go | 270 -------------- pulsaradmin/pkg/auth/oauth2_test.go | 48 ++- pulsaradmin/pkg/pulsar/admin.go | 15 + pulsaradmin/pkg/pulsar/subscription.go | 1 + pulsaradmin/pkg/pulsar/utils/message.go | 1 + 20 files changed, 212 insertions(+), 1899 deletions(-) delete mode 100644 pulsaradmin/pkg/auth/oauth2/auth.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/auth_suite_test.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever_test.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/client_credentials_flow.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/client_credentials_flow_test.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/client_credentials_provider.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/config_tokenprovider.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/config_tokenprovider_test.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider_test.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/plugin/clock.go delete mode 100644 pulsaradmin/pkg/auth/oauth2/plugin/fake_clock.go diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index d1665030f9..07ba304916 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -4,11 +4,12 @@ go 1.12 require ( github.com/99designs/keyring v1.1.5 + github.com/apache/pulsar-client-go/oauth2 v0.0.0-20200715083626-b9f8c5cedefb github.com/dgrijalva/jwt-go v3.2.0+incompatible github.com/docker/go-connections v0.4.0 github.com/fatih/color v1.7.0 github.com/ghodss/yaml v1.0.0 - github.com/golang/protobuf v1.3.2 + github.com/golang/protobuf v1.4.2 github.com/imdario/mergo v0.3.8 github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b @@ -16,15 +17,14 @@ require ( github.com/mattn/go-colorable v0.1.2 // indirect github.com/mattn/go-runewidth v0.0.4 // indirect github.com/olekukonko/tablewriter v0.0.1 - github.com/onsi/ginkgo v1.11.0 - github.com/onsi/gomega v1.7.0 - github.com/pkg/errors v0.8.1 + github.com/pkg/errors v0.9.1 github.com/spf13/cobra v0.0.5 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.4.0 github.com/testcontainers/testcontainers-go v0.0.10 - golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550 // indirect - golang.org/x/oauth2 v0.0.0-20190402181905-9f3314589c9a - golang.org/x/sync v0.0.0-20190423024810-112230192c58 // indirect - gopkg.in/yaml.v2 v2.2.8 + golang.org/x/net v0.0.0-20200707034311-ab3426394381 // indirect + golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d + google.golang.org/appengine v1.6.6 // indirect + google.golang.org/protobuf v1.25.0 // indirect + gopkg.in/yaml.v2 v2.3.0 ) diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index cb252d815b..ade68bc1e3 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -56,12 +56,10 @@ func GetAuthProvider(config *common.Config) (*Provider, error) { case len(config.TokenFile) > 0: provider, err = NewAuthenticationTokenFromFile(config.TokenFile, defaultTransport) case len(config.IssuerEndpoint) > 0 || len(config.KeyFile) > 0: - provider, err = NewAuthenticationOAuth2( + provider, err = NewAuthenticationOAuth2WithParams( config.IssuerEndpoint, config.ClientID, - config.Audience, - config.KeyFile, - defaultTransport) + config.Audience) } } return &provider, err diff --git a/pulsaradmin/pkg/auth/oauth2.go b/pulsaradmin/pkg/auth/oauth2.go index 1896b287d2..00a50b745a 100644 --- a/pulsaradmin/pkg/auth/oauth2.go +++ b/pulsaradmin/pkg/auth/oauth2.go @@ -18,77 +18,180 @@ package auth import ( - "errors" "fmt" "net/http" + "path/filepath" - "github.com/streamnative/pulsar-admin-go/pkg/auth/oauth2" + "github.com/99designs/keyring" + "github.com/apache/pulsar-client-go/oauth2" + "github.com/apache/pulsar-client-go/oauth2/cache" + clock2 "github.com/apache/pulsar-client-go/oauth2/clock" + "github.com/apache/pulsar-client-go/oauth2/store" + util "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + xoauth2 "golang.org/x/oauth2" +) + +const ( + TypeClientCredential = "client_credentials" + TypeDeviceCode = "device_code" ) type OAuth2Provider struct { - issuer *oauth2.Issuer - keyFile string - T http.RoundTripper + clock clock2.RealClock + issuer oauth2.Issuer + store store.Store + source cache.CachingTokenSource + T http.RoundTripper } -func NewAuthenticationOAuth2( +func NewAuthenticationOauth2(issuer oauth2.Issuer, store store.Store) (*OAuth2Provider, error) { + p := &OAuth2Provider{ + clock: clock2.RealClock{}, + issuer: issuer, + store: store, + } + + err := p.loadGrant() + if err != nil { + return nil, err + } + + return p, nil +} + +func NewAuthenticationOAuth2WithParams( issueEndpoint, clientID, - audience, - keyFile string, - transport http.RoundTripper) (*OAuth2Provider, error) { - - return &OAuth2Provider{ - issuer: &oauth2.Issuer{ - IssuerEndpoint: issueEndpoint, - ClientID: clientID, - Audience: audience, - }, - keyFile: keyFile, - T: transport, - }, nil -} + audience string) (*OAuth2Provider, error) { -func (o *OAuth2Provider) RoundTrip(req *http.Request) (*http.Response, error) { - token, err := o.getToken(o.issuer) + issuer := oauth2.Issuer{ + IssuerEndpoint: issueEndpoint, + ClientID: clientID, + Audience: audience, + } + + keyringStore, err := MakeKeyringStore() if err != nil { return nil, err } - req.Header.Add("Authorization", fmt.Sprintf("Bearer %s", token)) - return o.T.RoundTrip(req) + + p := &OAuth2Provider{ + clock: clock2.RealClock{}, + issuer: issuer, + store: keyringStore, + } + + err = p.loadGrant() + if err != nil { + return nil, err + } + + return p, nil +} + +func (o *OAuth2Provider) loadGrant() error { + grant, err := o.store.LoadGrant(o.issuer.Audience) + if err != nil { + return err + } + return o.initCache(grant) +} + +func (o *OAuth2Provider) initCache(grant *oauth2.AuthorizationGrant) error { + refresher, err := o.getRefresher(grant.Type) + if err != nil { + return err + } + + source, err := cache.NewDefaultTokenCache(o.store, o.issuer.Audience, refresher) + if err != nil { + return err + } + o.source = source + return nil +} + +func (o *OAuth2Provider) RoundTrip(req *http.Request) (*http.Response, error) { + return o.Transport().RoundTrip(req) } func (o *OAuth2Provider) Transport() http.RoundTripper { - return o.T + return &transport{ + source: o.source, + wrapped: &xoauth2.Transport{ + Source: o.source, + Base: o.T, + }, + } } -func (o *OAuth2Provider) getFlow(issuer *oauth2.Issuer) (oauth2.Flow, error) { - // note that these flows don't rely on the user's cache or configuration - // to produce an ephemeral token that doesn't replace what is generated - // by `login` or by `activate-service-account`. +func (o *OAuth2Provider) getRefresher(t oauth2.AuthorizationGrantType) (oauth2.AuthorizationGrantRefresher, error) { + switch t { + case oauth2.GrantTypeClientCredentials: + return oauth2.NewDefaultClientCredentialsGrantRefresher(o.clock) + case oauth2.GrantTypeDeviceCode: + return oauth2.NewDefaultDeviceAuthorizationGrantRefresher(o.clock) + default: + return nil, store.ErrUnsupportedAuthData + } +} + +type transport struct { + source cache.CachingTokenSource + wrapped *xoauth2.Transport +} - var err error - var flow oauth2.Flow - if o.keyFile != "" { - flow, err = oauth2.NewDefaultClientCredentialsFlow(*issuer, o.keyFile) +func (t *transport) RoundTrip(req *http.Request) (*http.Response, error) { + if len(req.Header.Get("Authorization")) != 0 { + return t.wrapped.Base.RoundTrip(req) + } + + token, err := t.source.Token() + if err != nil { + fmt.Println(err.Error()) + } + fmt.Println(token.AccessToken) + res, err := t.wrapped.RoundTrip(req) + if err != nil { + return nil, err + } + + if res.StatusCode == 401 { + err := t.source.InvalidateToken() if err != nil { return nil, err } - return flow, err } - return flow, errors.New("the key file must be specified") + + return res, nil } -func (o *OAuth2Provider) getToken(issuer *oauth2.Issuer) (string, error) { - flow, err := o.getFlow(issuer) - if err != nil { - return "", err - } +func (t *transport) WrappedRoundTripper() http.RoundTripper { return t.wrapped.Base } + +const ( + serviceName = "pulsar" + keyChainName = "pulsarctl" +) - _, token, err := flow.Authorize() +func MakeKeyringStore() (store.Store, error) { + kr, err := makeKeyring() if err != nil { - return "", err + return nil, err } + return store.NewKeyringStore(kr) +} + +func makeKeyring() (keyring.Keyring, error) { + return keyring.Open(keyring.Config{ + AllowedBackends: keyring.AvailableBackends(), + ServiceName: serviceName, + KeychainName: keyChainName, + KeychainTrustApplication: true, + FileDir: filepath.Join(util.HomeDir(), "~/.config/pulsar", "credentials"), + FilePasswordFunc: keyringPrompt, + }) +} - return token.AccessToken, nil +func keyringPrompt(prompt string) (string, error) { + return "", nil } diff --git a/pulsaradmin/pkg/auth/oauth2/auth.go b/pulsaradmin/pkg/auth/oauth2/auth.go deleted file mode 100644 index d6d99f04b5..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/auth.go +++ /dev/null @@ -1,88 +0,0 @@ -// 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 oauth2 - -import ( - "fmt" - "time" - - "github.com/dgrijalva/jwt-go" - "github.com/streamnative/pulsar-admin-go/pkg/auth/oauth2/plugin" - "golang.org/x/oauth2" -) - -const ( - ClaimNameUserName = "https://pulsar.apache.org/username" -) - -// Flow abstracts an OAuth 2.0 authentication and authorization flow -type Flow interface { - // Authorize obtains an authorization grant based on an OAuth 2.0 authorization flow. - // The method returns a grant and (optionally) an initial access token. - Authorize() (AuthorizationGrant, *oauth2.Token, error) -} - -// AuthorizationGrant is a credential representing the resource owner's authorization -// to access its protected resources, used by the client to obtain an access token -type AuthorizationGrant interface { - // Refresh obtains a fresh access token based on this grant - Refresh() (*oauth2.Token, error) -} - -// TokenResult holds token information -type TokenResult struct { - AccessToken string `json:"access_token"` - IDToken string `json:"id_token"` - RefreshToken string `json:"refresh_token"` - ExpiresIn int `json:"expires_in"` -} - -// Issuer holds information about the issuer of tokens -type Issuer struct { - IssuerEndpoint string - ClientID string - Audience string -} - -func convertToOAuth2Token(token *TokenResult, clock plugin.Clock) oauth2.Token { - return oauth2.Token{ - AccessToken: token.AccessToken, - TokenType: "bearer", - RefreshToken: token.RefreshToken, - Expiry: clock.Now().Add(time.Duration(token.ExpiresIn) * time.Second), - } -} - -// ExtractUserName extracts the username claim from an authorization grant -func ExtractUserName(token oauth2.Token) (string, error) { - p := jwt.Parser{} - claims := jwt.MapClaims{} - if _, _, err := p.ParseUnverified(token.AccessToken, claims); err != nil { - return "", fmt.Errorf("unable to decode the access token: %v", err) - } - username, ok := claims[ClaimNameUserName] - if !ok { - return "", fmt.Errorf("access token doesn't contain a username claim") - } - switch v := username.(type) { - case string: - return v, nil - default: - return "", fmt.Errorf("access token contains an unsupported username claim") - } -} diff --git a/pulsaradmin/pkg/auth/oauth2/auth_suite_test.go b/pulsaradmin/pkg/auth/oauth2/auth_suite_test.go deleted file mode 100644 index d35282b52e..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/auth_suite_test.go +++ /dev/null @@ -1,59 +0,0 @@ -// 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 oauth2 - -import ( - "context" - "testing" - - . "github.com/onsi/ginkgo" - . "github.com/onsi/gomega" -) - -func TestAuth(t *testing.T) { - RegisterFailHandler(Fail) - RunSpecs(t, "Pulsarctl Auth Suite") -} - -type MockTokenExchanger struct { - CalledWithRequest interface{} - ReturnsTokens *TokenResult - ReturnsError error - RefreshCalledWithRequest *RefreshTokenExchangeRequest -} - -func (te *MockTokenExchanger) ExchangeCode(req AuthorizationCodeExchangeRequest) (*TokenResult, error) { - te.CalledWithRequest = &req - return te.ReturnsTokens, te.ReturnsError -} - -func (te *MockTokenExchanger) ExchangeRefreshToken(req RefreshTokenExchangeRequest) (*TokenResult, error) { - te.RefreshCalledWithRequest = &req - return te.ReturnsTokens, te.ReturnsError -} - -func (te *MockTokenExchanger) ExchangeClientCredentials(req ClientCredentialsExchangeRequest) (*TokenResult, error) { - te.CalledWithRequest = &req - return te.ReturnsTokens, te.ReturnsError -} - -func (te *MockTokenExchanger) ExchangeDeviceCode(ctx context.Context, - req DeviceCodeExchangeRequest) (*TokenResult, error) { - te.CalledWithRequest = &req - return te.ReturnsTokens, te.ReturnsError -} diff --git a/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever.go b/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever.go deleted file mode 100644 index 4723482456..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever.go +++ /dev/null @@ -1,338 +0,0 @@ -// 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 oauth2 - -import ( - "context" - "encoding/json" - "errors" - "fmt" - "net/http" - "net/url" - "strconv" - "strings" - "time" -) - -// TokenRetriever implements AuthTokenExchanger in order to facilitate getting -// Tokens -type TokenRetriever struct { - oidcWellKnownEndpoints OIDCWellKnownEndpoints - transport HTTPAuthTransport -} - -// AuthorizationTokenResponse is the HTTP response when asking for a new token. -// Note that not all fields will contain data based on what kind of request was -// sent -type AuthorizationTokenResponse struct { - AccessToken string `json:"access_token"` - ExpiresIn int `json:"expires_in"` - IDToken string `json:"id_token"` - RefreshToken string `json:"refresh_token"` - TokenType string `json:"token_type"` -} - -// AuthorizationCodeExchangeRequest is used to request the exchange of an -// authorization code for a token -type AuthorizationCodeExchangeRequest struct { - ClientID string - CodeVerifier string - Code string - RedirectURI string -} - -// RefreshTokenExchangeRequest is used to request the exchange of a refresh -// token for a refreshed token -type RefreshTokenExchangeRequest struct { - ClientID string - RefreshToken string -} - -// ClientCredentialsExchangeRequest is used to request the exchange of -// client credentials for a token -type ClientCredentialsExchangeRequest struct { - ClientID string - ClientSecret string - Audience string -} - -// DeviceCodeExchangeRequest is used to request the exchange of -// a device code for a token -type DeviceCodeExchangeRequest struct { - ClientID string - DeviceCode string - PollInterval time.Duration -} - -// TokenErrorResponse is used to parse error responses from the token endpoint -type TokenErrorResponse struct { - Error string `json:"error"` - ErrorDescription string `json:"error_description"` -} - -type TokenError struct { - ErrorCode string - ErrorDescription string -} - -func (e *TokenError) Error() string { - if e.ErrorDescription != "" { - return fmt.Sprintf("%s (%s)", e.ErrorDescription, e.ErrorCode) - } - return e.ErrorCode -} - -// HTTPAuthTransport abstracts how an HTTP exchange request is sent and received -type HTTPAuthTransport interface { - Do(request *http.Request) (*http.Response, error) -} - -// NewTokenRetriever allows a TokenRetriever the internal of a new -// TokenRetriever to be easily set up -func NewTokenRetriever( - oidcWellKnownEndpoints OIDCWellKnownEndpoints, - authTransport HTTPAuthTransport) *TokenRetriever { - return &TokenRetriever{ - oidcWellKnownEndpoints: oidcWellKnownEndpoints, - transport: authTransport, - } -} - -// newExchangeCodeRequest builds a new AuthTokenRequest wrapped in an -// http.Request -func (ce *TokenRetriever) newExchangeCodeRequest( - req AuthorizationCodeExchangeRequest) (*http.Request, error) { - uv := url.Values{} - uv.Set("grant_type", "authorization_code") - uv.Set("client_id", req.ClientID) - uv.Set("code_verifier", req.CodeVerifier) - uv.Set("code", req.Code) - uv.Set("redirect_uri", req.RedirectURI) - - euv := uv.Encode() - - request, err := http.NewRequest("POST", - ce.oidcWellKnownEndpoints.TokenEndpoint, - strings.NewReader(euv), - ) - if err != nil { - return nil, err - } - - request.Header.Add("Content-Type", "application/x-www-form-urlencoded") - request.Header.Add("Content-Length", strconv.Itoa(len(euv))) - - return request, nil -} - -// newDeviceCodeExchangeRequest builds a new DeviceCodeExchangeRequest wrapped in an -// http.Request -func (ce *TokenRetriever) newDeviceCodeExchangeRequest( - req DeviceCodeExchangeRequest) (*http.Request, error) { - uv := url.Values{} - uv.Set("grant_type", "urn:ietf:params:oauth:grant-type:device_code") - uv.Set("client_id", req.ClientID) - uv.Set("device_code", req.DeviceCode) - euv := uv.Encode() - - request, err := http.NewRequest("POST", - ce.oidcWellKnownEndpoints.TokenEndpoint, - strings.NewReader(euv), - ) - if err != nil { - return nil, err - } - - request.Header.Add("Content-Type", "application/x-www-form-urlencoded") - request.Header.Add("Content-Length", strconv.Itoa(len(euv))) - - return request, nil -} - -// newRefreshTokenRequest builds a new RefreshTokenRequest wrapped in an -// http.Request -func (ce *TokenRetriever) newRefreshTokenRequest(req RefreshTokenExchangeRequest) (*http.Request, error) { - uv := url.Values{} - uv.Set("grant_type", "refresh_token") - uv.Set("client_id", req.ClientID) - uv.Set("refresh_token", req.RefreshToken) - - euv := uv.Encode() - - request, err := http.NewRequest("POST", - ce.oidcWellKnownEndpoints.TokenEndpoint, - strings.NewReader(euv), - ) - if err != nil { - return nil, err - } - - request.Header.Add("Content-Type", "application/x-www-form-urlencoded") - request.Header.Add("Content-Length", strconv.Itoa(len(euv))) - - return request, nil -} - -// newClientCredentialsRequest builds a new ClientCredentialsExchangeRequest wrapped in an -// http.Request -func (ce *TokenRetriever) newClientCredentialsRequest(req ClientCredentialsExchangeRequest) (*http.Request, error) { - uv := url.Values{} - uv.Set("grant_type", "client_credentials") - uv.Set("client_id", req.ClientID) - uv.Set("client_secret", req.ClientSecret) - uv.Set("audience", req.Audience) - - euv := uv.Encode() - - request, err := http.NewRequest("POST", - ce.oidcWellKnownEndpoints.TokenEndpoint, - strings.NewReader(euv), - ) - if err != nil { - return nil, err - } - - request.Header.Add("Content-Type", "application/x-www-form-urlencoded") - request.Header.Add("Content-Length", strconv.Itoa(len(euv))) - - return request, nil -} - -// ExchangeCode uses the AuthCodeExchangeRequest to exchange an authorization -// code for tokens -func (ce *TokenRetriever) ExchangeCode(req AuthorizationCodeExchangeRequest) (*TokenResult, error) { - request, err := ce.newExchangeCodeRequest(req) - if err != nil { - return nil, err - } - - response, err := ce.transport.Do(request) - if err != nil { - return nil, err - } - - return ce.handleAuthTokensResponse(response) -} - -// handleAuthTokensResponse takes care of checking an http.Response that has -// auth tokens for errors and parsing the raw body to a TokenResult struct -func (ce *TokenRetriever) handleAuthTokensResponse(resp *http.Response) (*TokenResult, error) { - if resp.Body != nil { - defer resp.Body.Close() - } - - if resp.StatusCode < 200 || resp.StatusCode > 299 { - if resp.Header.Get("Content-Type") == "application/json" { - er := TokenErrorResponse{} - err := json.NewDecoder(resp.Body).Decode(&er) - if err != nil { - return nil, err - } - return nil, &TokenError{ErrorCode: er.Error, ErrorDescription: er.ErrorDescription} - } - return nil, fmt.Errorf("a non-success status code was received: %d", resp.StatusCode) - } - - atr := AuthorizationTokenResponse{} - err := json.NewDecoder(resp.Body).Decode(&atr) - if err != nil { - return nil, err - } - - return &TokenResult{ - AccessToken: atr.AccessToken, - IDToken: atr.IDToken, - RefreshToken: atr.RefreshToken, - ExpiresIn: atr.ExpiresIn, - }, nil -} - -// ExchangeDeviceCode uses the DeviceCodeExchangeRequest to exchange a device -// code for tokens -func (ce *TokenRetriever) ExchangeDeviceCode(ctx context.Context, req DeviceCodeExchangeRequest) (*TokenResult, error) { - for { - request, err := ce.newDeviceCodeExchangeRequest(req) - if err != nil { - return nil, err - } - - response, err := ce.transport.Do(request) - if err != nil { - return nil, err - } - token, err := ce.handleAuthTokensResponse(response) - if err == nil { - return token, nil - } - terr, ok := err.(*TokenError) - if !ok { - return nil, err - } - switch terr.ErrorCode { - case "expired_token": - // The user has not authorized the device quickly enough, so the device_code has expired. - return nil, fmt.Errorf("the device code has expired") - case "access_denied": - // The user refused to authorize the device - return nil, fmt.Errorf("the device was not authorized") - case "authorization_pending": - // Still waiting for the user to take action - case "slow_down": - // You are polling too fast - } - - select { - case <-time.After(req.PollInterval): - continue - case <-ctx.Done(): - return nil, errors.New("cancelled") - } - } -} - -// ExchangeRefreshToken uses the RefreshTokenExchangeRequest to exchange a -// refresh token for refreshed tokens -func (ce *TokenRetriever) ExchangeRefreshToken(req RefreshTokenExchangeRequest) (*TokenResult, error) { - request, err := ce.newRefreshTokenRequest(req) - if err != nil { - return nil, err - } - - response, err := ce.transport.Do(request) - if err != nil { - return nil, err - } - - return ce.handleAuthTokensResponse(response) -} - -// ExchangeClientCredentials uses the ClientCredentialsExchangeRequest to exchange -// client credentials for tokens -func (ce *TokenRetriever) ExchangeClientCredentials(req ClientCredentialsExchangeRequest) (*TokenResult, error) { - request, err := ce.newClientCredentialsRequest(req) - if err != nil { - return nil, err - } - - response, err := ce.transport.Do(request) - if err != nil { - return nil, err - } - - return ce.handleAuthTokensResponse(response) -} diff --git a/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever_test.go b/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever_test.go deleted file mode 100644 index affe255244..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/authorization_tokenretriever_test.go +++ /dev/null @@ -1,334 +0,0 @@ -// 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 oauth2 - -import ( - "bytes" - "context" - "encoding/json" - "io/ioutil" - "net/http" - "strings" - "time" - - . "github.com/onsi/ginkgo" - . "github.com/onsi/gomega" -) - -type MockTransport struct { - Responses []*http.Response - ReturnError error -} - -var _ HTTPAuthTransport = &MockTransport{} - -func (t *MockTransport) Do(req *http.Request) (*http.Response, error) { - if len(t.Responses) > 0 { - r := t.Responses[0] - t.Responses = t.Responses[1:] - return r, nil - } - return nil, t.ReturnError -} - -var _ = Describe("CodetokenExchanger", func() { - Describe("newExchangeCodeRequest", func() { - It("creates the request", func() { - tokenRetriever := TokenRetriever{ - oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "https://issuer/oauth/token"}} - exchangeRequest := AuthorizationCodeExchangeRequest{ - ClientID: "clientID", - CodeVerifier: "Verifier", - Code: "code", - RedirectURI: "https://redirect", - } - - result, err := tokenRetriever.newExchangeCodeRequest(exchangeRequest) - - result.ParseForm() - - Expect(err).To(BeNil()) - Expect(result.FormValue("grant_type")).To(Equal("authorization_code")) - Expect(result.FormValue("client_id")).To(Equal("clientID")) - Expect(result.FormValue("code_verifier")).To(Equal("Verifier")) - Expect(result.FormValue("code")).To(Equal("code")) - Expect(result.FormValue("redirect_uri")).To(Equal("https://redirect")) - Expect(result.URL.String()).To(Equal("https://issuer/oauth/token")) - - Expect(result.Header.Get("Content-Type")).To(Equal("application/x-www-form-urlencoded")) - Expect(result.Header.Get("Content-Length")).To(Equal("117")) - }) - - It("returns an error when NewRequest returns an error", func() { - tokenRetriever := TokenRetriever{ - oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "://issuer/oauth/token"}} - - result, err := tokenRetriever.newExchangeCodeRequest(AuthorizationCodeExchangeRequest{}) - - Expect(result).To(BeNil()) - Expect(err.Error()).To(Equal("parse ://issuer/oauth/token: missing protocol scheme")) - }) - }) - - Describe("handleAuthTokensResponse", func() { - It("handles the response", func() { - tokenRetriever := TokenRetriever{} - response := buildResponse(200, AuthorizationTokenResponse{ - ExpiresIn: 1, - AccessToken: "myAccessToken", - RefreshToken: "myRefreshToken", - }) - - result, err := tokenRetriever.handleAuthTokensResponse(response) - - Expect(err).To(BeNil()) - Expect(result).To(Equal(&TokenResult{ - ExpiresIn: 1, - AccessToken: "myAccessToken", - RefreshToken: "myRefreshToken", - })) - }) - - It("returns error when status code is not successful", func() { - tokenRetriever := TokenRetriever{} - response := buildResponse(500, nil) - - result, err := tokenRetriever.handleAuthTokensResponse(response) - - Expect(result).To(BeNil()) - Expect(err.Error()).To(Equal("a non-success status code was received: 500")) - }) - - It("returns typed error when response body contains error information", func() { - errorBody := TokenErrorResponse{Error: "test", ErrorDescription: "test description"} - tokenRetriever := TokenRetriever{} - response := buildResponse(400, errorBody) - - result, err := tokenRetriever.handleAuthTokensResponse(response) - - Expect(result).To(BeNil()) - Expect(err).To(Equal(&TokenError{ErrorCode: "test", ErrorDescription: "test description"})) - Expect(err.Error()).To(Equal("test description (test)")) - }) - - It("returns error when deserialization fails", func() { - tokenRetriever := TokenRetriever{} - response := buildResponse(200, "") - - result, err := tokenRetriever.handleAuthTokensResponse(response) - Expect(result).To(BeNil()) - Expect(err.Error()).To(Equal( - "json: cannot unmarshal string into Go value of type oauth2.AuthorizationTokenResponse")) - }) - }) - - Describe("newRefreshTokenRequest", func() { - It("creates the request", func() { - tokenRetriever := TokenRetriever{ - oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "https://issuer/oauth/token"}} - exchangeRequest := RefreshTokenExchangeRequest{ - ClientID: "clientID", - RefreshToken: "refreshToken", - } - - result, err := tokenRetriever.newRefreshTokenRequest(exchangeRequest) - - result.ParseForm() - - Expect(err).To(BeNil()) - Expect(result.FormValue("grant_type")).To(Equal("refresh_token")) - Expect(result.FormValue("client_id")).To(Equal("clientID")) - Expect(result.FormValue("refresh_token")).To(Equal("refreshToken")) - Expect(result.URL.String()).To(Equal("https://issuer/oauth/token")) - - Expect(result.Header.Get("Content-Type")).To(Equal("application/x-www-form-urlencoded")) - Expect(result.Header.Get("Content-Length")).To(Equal("70")) - }) - - It("returns an error when NewRequest returns an error", func() { - tokenRetriever := TokenRetriever{ - oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "://issuer/oauth/token"}} - - result, err := tokenRetriever.newRefreshTokenRequest(RefreshTokenExchangeRequest{}) - - Expect(result).To(BeNil()) - Expect(err.Error()).To(Equal("parse ://issuer/oauth/token: missing protocol scheme")) - }) - }) - - Describe("newClientCredentialsRequest", func() { - It("creates the request", func() { - tokenRetriever := TokenRetriever{ - oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "https://issuer/oauth/token"}} - exchangeRequest := ClientCredentialsExchangeRequest{ - ClientID: "clientID", - ClientSecret: "clientSecret", - Audience: "audience", - } - - result, err := tokenRetriever.newClientCredentialsRequest(exchangeRequest) - - result.ParseForm() - - Expect(err).To(BeNil()) - Expect(result.FormValue("grant_type")).To(Equal("client_credentials")) - Expect(result.FormValue("client_id")).To(Equal("clientID")) - Expect(result.FormValue("client_secret")).To(Equal("clientSecret")) - Expect(result.FormValue("audience")).To(Equal("audience")) - Expect(result.URL.String()).To(Equal("https://issuer/oauth/token")) - - Expect(result.Header.Get("Content-Type")).To(Equal("application/x-www-form-urlencoded")) - Expect(result.Header.Get("Content-Length")).To(Equal("93")) - }) - - It("returns an error when NewRequest returns an error", func() { - tokenRetriever := TokenRetriever{ - oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "://issuer/oauth/token"}} - - result, err := tokenRetriever.newClientCredentialsRequest(ClientCredentialsExchangeRequest{}) - - Expect(result).To(BeNil()) - Expect(err.Error()).To(Equal("parse ://issuer/oauth/token: missing protocol scheme")) - }) - }) - - Describe("newDeviceCodeExchangeRequest", func() { - It("creates the request", func() { - tokenRetriever := TokenRetriever{ - oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "https://issuer/oauth/token"}} - exchangeRequest := DeviceCodeExchangeRequest{ - ClientID: "clientID", - DeviceCode: "deviceCode", - PollInterval: time.Duration(5) * time.Second, - } - - result, err := tokenRetriever.newDeviceCodeExchangeRequest(exchangeRequest) - - result.ParseForm() - - Expect(err).To(BeNil()) - Expect(result.FormValue("grant_type")).To(Equal("urn:ietf:params:oauth:grant-type:device_code")) - Expect(result.FormValue("client_id")).To(Equal("clientID")) - Expect(result.FormValue("device_code")).To(Equal("deviceCode")) - Expect(result.URL.String()).To(Equal("https://issuer/oauth/token")) - - Expect(result.Header.Get("Content-Type")).To(Equal("application/x-www-form-urlencoded")) - Expect(result.Header.Get("Content-Length")).To(Equal("107")) - }) - - It("returns an error when NewRequest returns an error", func() { - tokenRetriever := TokenRetriever{ - oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "://issuer/oauth/token"}} - - result, err := tokenRetriever.newClientCredentialsRequest(ClientCredentialsExchangeRequest{}) - - Expect(result).To(BeNil()) - Expect(err.Error()).To(Equal("parse ://issuer/oauth/token: missing protocol scheme")) - }) - }) - - Describe("ExchangeDeviceCode", func() { - var mockTransport *MockTransport - var tokenRetriever *TokenRetriever - var exchangeRequest DeviceCodeExchangeRequest - var tokenResult TokenResult - - BeforeEach(func() { - mockTransport = &MockTransport{} - tokenRetriever = &TokenRetriever{ - oidcWellKnownEndpoints: OIDCWellKnownEndpoints{TokenEndpoint: "https://issuer/oauth/token"}, - transport: mockTransport, - } - exchangeRequest = DeviceCodeExchangeRequest{ - ClientID: "clientID", - DeviceCode: "deviceCode", - PollInterval: time.Duration(1) * time.Second, - } - tokenResult = TokenResult{ - ExpiresIn: 1, - AccessToken: "myAccessToken", - RefreshToken: "myRefreshToken", - } - }) - - It("returns a token", func() { - }) - - It("supports cancellation", func() { - mockTransport.Responses = []*http.Response{ - buildResponse(400, &TokenErrorResponse{"authorization_pending", ""}), - } - ctx, cancel := context.WithCancel(context.Background()) - cancel() - _, err := tokenRetriever.ExchangeDeviceCode(ctx, exchangeRequest) - Expect(err).ToNot(BeNil()) - Expect(err.Error()).To(Equal("cancelled")) - }) - - It("implements authorization_pending and slow_down", func() { - startTime := time.Now() - mockTransport.Responses = []*http.Response{ - buildResponse(400, &TokenErrorResponse{"authorization_pending", ""}), - buildResponse(400, &TokenErrorResponse{"authorization_pending", ""}), - buildResponse(400, &TokenErrorResponse{"slow_down", ""}), - buildResponse(200, &tokenResult), - } - token, err := tokenRetriever.ExchangeDeviceCode(context.Background(), exchangeRequest) - Expect(err).To(BeNil()) - Expect(token).To(Equal(&tokenResult)) - endTime := time.Now() - Expect(endTime.Sub(startTime)).To(BeNumerically(">", exchangeRequest.PollInterval*3)) - }) - - It("implements expired_token", func() { - mockTransport.Responses = []*http.Response{ - buildResponse(400, &TokenErrorResponse{"expired_token", ""}), - } - _, err := tokenRetriever.ExchangeDeviceCode(context.Background(), exchangeRequest) - Expect(err).ToNot(BeNil()) - Expect(err.Error()).To(Equal("the device code has expired")) - }) - - It("implements access_denied", func() { - mockTransport.Responses = []*http.Response{ - buildResponse(400, &TokenErrorResponse{"access_denied", ""}), - } - _, err := tokenRetriever.ExchangeDeviceCode(context.Background(), exchangeRequest) - Expect(err).ToNot(BeNil()) - Expect(err.Error()).To(Equal("the device was not authorized")) - }) - }) -}) - -func buildResponse(statusCode int, body interface{}) *http.Response { - b, err := json.Marshal(body) - if err != nil { - panic(err) - } - - resp := &http.Response{ - StatusCode: statusCode, - Header: map[string][]string{}, - Body: ioutil.NopCloser(bytes.NewReader(b)), - } - if strings.HasPrefix(string(b), "{") { - resp.Header.Add("Content-Type", "application/json") - } - - return resp -} diff --git a/pulsaradmin/pkg/auth/oauth2/client_credentials_flow.go b/pulsaradmin/pkg/auth/oauth2/client_credentials_flow.go deleted file mode 100644 index ba1a187ce0..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/client_credentials_flow.go +++ /dev/null @@ -1,149 +0,0 @@ -// 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 oauth2 - -import ( - "net/http" - - "github.com/streamnative/pulsar-admin-go/pkg/auth/oauth2/plugin" - "golang.org/x/oauth2" - - "github.com/pkg/errors" -) - -// ClientCredentialsFlow takes care of the mechanics needed for getting an access -// token using the OAuth 2.0 "Client Credentials Flow" -type ClientCredentialsFlow struct { - issuerData Issuer - provider ClientCredentialsProvider - exchanger ClientCredentialsExchanger - clock plugin.Clock -} - -// ClientCredentialsProvider abstracts getting client credentials -type ClientCredentialsProvider interface { - GetClientCredentials() (*KeyFile, error) -} - -// ClientCredentialsExchanger abstracts exchanging client credentials for tokens -type ClientCredentialsExchanger interface { - ExchangeClientCredentials(req ClientCredentialsExchangeRequest) (*TokenResult, error) -} - -func NewClientCredentialsFlow( - issuerData Issuer, - provider ClientCredentialsProvider, - exchanger ClientCredentialsExchanger, - clock plugin.Clock) *ClientCredentialsFlow { - return &ClientCredentialsFlow{ - issuerData: issuerData, - provider: provider, - exchanger: exchanger, - clock: clock, - } -} - -// NewDefaultClientCredentialsFlow provides an easy way to build up a default -// client credentials flow with all the correct configuration. -func NewDefaultClientCredentialsFlow(issuerData Issuer, keyFile string) (*ClientCredentialsFlow, error) { - wellKnownEndpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL(issuerData.IssuerEndpoint) - if err != nil { - return nil, err - } - - credsProvider := NewClientCredentialsProviderFromKeyFile(keyFile) - - tokenRetriever := NewTokenRetriever( - *wellKnownEndpoints, - &http.Client{}) - - return NewClientCredentialsFlow( - issuerData, - credsProvider, - tokenRetriever, - plugin.RealClock{}), nil -} - -var _ Flow = &ClientCredentialsFlow{} - -func (c *ClientCredentialsFlow) Authorize() (AuthorizationGrant, *oauth2.Token, error) { - keyFile, err := c.provider.GetClientCredentials() - if err != nil { - return nil, nil, errors.Wrap(err, "could not get client credentials") - } - - grant := &ClientCredentialsGrant{ - KeyFile: *keyFile, - issuerData: c.issuerData, - exchanger: c.exchanger, - clock: c.clock, - } - - // test the credentials and obtain an initial access token - token, err := grant.Refresh() - if err != nil { - return nil, nil, errors.Wrap(err, "authentication failed using client credentials") - } - - return grant, token, nil -} - -type ClientCredentialsGrant struct { - KeyFile KeyFile - issuerData Issuer - exchanger ClientCredentialsExchanger - clock plugin.Clock -} - -func NewDefaultClientCredentialsGrant(issuerData Issuer, keyFile KeyFile, - clock plugin.Clock) (*ClientCredentialsGrant, error) { - wellKnownEndpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL(issuerData.IssuerEndpoint) - if err != nil { - return nil, err - } - - tokenRetriever := NewTokenRetriever( - *wellKnownEndpoints, - &http.Client{}) - - return &ClientCredentialsGrant{ - KeyFile: keyFile, - issuerData: issuerData, - exchanger: tokenRetriever, - clock: clock, - }, nil -} - -var _ AuthorizationGrant = &ClientCredentialsGrant{} - -func (g *ClientCredentialsGrant) Refresh() (*oauth2.Token, error) { - exchangeRequest := ClientCredentialsExchangeRequest{ - Audience: g.issuerData.Audience, - ClientID: g.KeyFile.ClientID, - ClientSecret: g.KeyFile.ClientSecret, - } - - tr, err := g.exchanger.ExchangeClientCredentials(exchangeRequest) - if err != nil { - return nil, errors.Wrap(err, "could not exchange client credentials") - } - - token := convertToOAuth2Token(tr, g.clock) - - return &token, nil -} diff --git a/pulsaradmin/pkg/auth/oauth2/client_credentials_flow_test.go b/pulsaradmin/pkg/auth/oauth2/client_credentials_flow_test.go deleted file mode 100644 index 6ac4be4064..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/client_credentials_flow_test.go +++ /dev/null @@ -1,135 +0,0 @@ -// 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 oauth2 - -import ( - "errors" - "time" - - . "github.com/onsi/ginkgo" - . "github.com/onsi/gomega" - "github.com/streamnative/pulsar-admin-go/pkg/auth/oauth2/plugin" -) - -type MockClientCredentialsProvider struct { - Called bool - ClientCredentialsResult *KeyFile - ReturnsError error -} - -func (m *MockClientCredentialsProvider) GetClientCredentials() (*KeyFile, error) { - m.Called = true - return m.ClientCredentialsResult, m.ReturnsError -} - -var _ ClientCredentialsProvider = &MockClientCredentialsProvider{} - -var _ = Describe("ClientCredentialsFlow", func() { - issuer := Issuer{ - IssuerEndpoint: "http://issuer", - ClientID: "", - Audience: "test_audience", - } - - Describe("Authorize", func() { - - var mockClock plugin.Clock - var mockCredsProvider *MockClientCredentialsProvider - var mockTokenExchanger *MockTokenExchanger - - BeforeEach(func() { - mockClock = plugin.NewFakeClock(time.Unix(0, 0)) - - mockCredsProvider = &MockClientCredentialsProvider{ - ClientCredentialsResult: &KeyFile{ - Type: KeyFileTypeServiceAccount, - ClientID: "test_clientID", - ClientSecret: "test_clientSecret", - ClientEmail: "test_clientEmail", - }, - } - - expectedTokens := TokenResult{AccessToken: "accessToken", RefreshToken: "refreshToken", ExpiresIn: 1234} - mockTokenExchanger = &MockTokenExchanger{ - ReturnsTokens: &expectedTokens, - } - }) - - It("invokes TokenExchanger with credentials", func() { - provider := NewClientCredentialsFlow( - issuer, - mockCredsProvider, - mockTokenExchanger, - mockClock, - ) - - _, _, err := provider.Authorize() - Expect(err).ToNot(HaveOccurred()) - Expect(mockCredsProvider.Called).To(BeTrue()) - Expect(mockTokenExchanger.CalledWithRequest).To(Equal(&ClientCredentialsExchangeRequest{ - ClientID: mockCredsProvider.ClientCredentialsResult.ClientID, - ClientSecret: mockCredsProvider.ClientCredentialsResult.ClientSecret, - Audience: issuer.Audience, - })) - }) - - It("returns TokensResult from TokenExchanger", func() { - provider := NewClientCredentialsFlow( - issuer, - mockCredsProvider, - mockTokenExchanger, - mockClock, - ) - - _, token, err := provider.Authorize() - Expect(err).ToNot(HaveOccurred()) - expected := convertToOAuth2Token(mockTokenExchanger.ReturnsTokens, mockClock) - Expect(*token).To(Equal(expected)) - }) - - It("returns an error if client credentials request errors", func() { - mockCredsProvider.ReturnsError = errors.New("someerror") - - provider := NewClientCredentialsFlow( - issuer, - mockCredsProvider, - mockTokenExchanger, - mockClock, - ) - - _, _, err := provider.Authorize() - Expect(err.Error()).To(Equal("could not get client credentials: someerror")) - }) - - It("returns an error if token exchanger errors", func() { - mockTokenExchanger.ReturnsError = errors.New("someerror") - mockTokenExchanger.ReturnsTokens = nil - - provider := NewClientCredentialsFlow( - issuer, - mockCredsProvider, - mockTokenExchanger, - mockClock, - ) - - _, _, err := provider.Authorize() - Expect(err.Error()).To(Equal("authentication failed using client credentials: " + - "could not exchange client credentials: someerror")) - }) - }) -}) diff --git a/pulsaradmin/pkg/auth/oauth2/client_credentials_provider.go b/pulsaradmin/pkg/auth/oauth2/client_credentials_provider.go deleted file mode 100644 index 48b2e46d39..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/client_credentials_provider.go +++ /dev/null @@ -1,65 +0,0 @@ -// 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 oauth2 - -import ( - "encoding/json" - "fmt" - "io/ioutil" -) - -const ( - KeyFileTypeServiceAccount = "sn_service_account" -) - -type KeyFileProvider struct { - KeyFile string -} - -type KeyFile struct { - Type string `json:"type"` - ClientID string `json:"client_id"` - ClientSecret string `json:"client_secret"` - ClientEmail string `json:"client_email"` -} - -func NewClientCredentialsProviderFromKeyFile(keyFile string) *KeyFileProvider { - return &KeyFileProvider{ - KeyFile: keyFile, - } -} - -var _ ClientCredentialsProvider = &KeyFileProvider{} - -func (k *KeyFileProvider) GetClientCredentials() (*KeyFile, error) { - keyFile, err := ioutil.ReadFile(k.KeyFile) - if err != nil { - return nil, err - } - - var v KeyFile - err = json.Unmarshal(keyFile, &v) - if err != nil { - return nil, err - } - if v.Type != KeyFileTypeServiceAccount { - return nil, fmt.Errorf("open %s: unsupported format", k.KeyFile) - } - - return &v, nil -} diff --git a/pulsaradmin/pkg/auth/oauth2/config_tokenprovider.go b/pulsaradmin/pkg/auth/oauth2/config_tokenprovider.go deleted file mode 100644 index 627749fb70..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/config_tokenprovider.go +++ /dev/null @@ -1,57 +0,0 @@ -// 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 oauth2 - -import "fmt" - -type configProvider interface { - GetTokens(identifier string) (string, string) - SaveTokens(identifier, accessToken, refreshToken string) -} - -// ConfigBackedCachingProvider wraps a configProvider in order to conform to -// the cachingProvider interface -type ConfigBackedCachingProvider struct { - identifier string - config configProvider -} - -// NewConfigBackedCachingProvider builds and returns a CachingTokenProvider -// that utilizes a configProvider to cache tokens -func NewConfigBackedCachingProvider(clientID, audience string, config configProvider) *ConfigBackedCachingProvider { - return &ConfigBackedCachingProvider{ - identifier: fmt.Sprintf("%s-%s", clientID, audience), - config: config, - } -} - -// GetTokens gets the tokens from the cache and returns them as a TokenResult -func (c *ConfigBackedCachingProvider) GetTokens() (*TokenResult, error) { - accessToken, refreshToken := c.config.GetTokens(c.identifier) - return &TokenResult{ - AccessToken: accessToken, - RefreshToken: refreshToken, - }, nil -} - -// CacheTokens caches the id and refresh token from TokenResult in the -// configProvider -func (c *ConfigBackedCachingProvider) CacheTokens(toCache *TokenResult) error { - c.config.SaveTokens(c.identifier, toCache.AccessToken, toCache.RefreshToken) - return nil -} diff --git a/pulsaradmin/pkg/auth/oauth2/config_tokenprovider_test.go b/pulsaradmin/pkg/auth/oauth2/config_tokenprovider_test.go deleted file mode 100644 index d949a5a5e9..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/config_tokenprovider_test.go +++ /dev/null @@ -1,91 +0,0 @@ -// 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 oauth2 - -import ( - . "github.com/onsi/ginkgo" - . "github.com/onsi/gomega" -) - -type mockConfigProvider struct { - ReturnAccessToken string - ReturnRefreshToken string - GetTokensCalledIdentifier string - SavedIdentifier string - SavedAccessToken string - SavedRefreshToken string -} - -func (m *mockConfigProvider) GetTokens(identifier string) (string, string) { - m.GetTokensCalledIdentifier = identifier - return m.ReturnAccessToken, m.ReturnRefreshToken -} - -func (m *mockConfigProvider) SaveTokens(identifier, accessToken, refreshToken string) { - m.SavedIdentifier = identifier - m.SavedAccessToken = accessToken - m.SavedRefreshToken = refreshToken -} - -var _ = Describe("main", func() { - Describe("configCachingProvider", func() { - It("sets up the identifier using the clientID and audience", func() { - p := NewConfigBackedCachingProvider("iamclientid", "iamaudience", &mockConfigProvider{}) - - Expect(p.identifier).To(Equal("iamclientid-iamaudience")) - }) - - It("gets tokens from the config provider", func() { - c := &mockConfigProvider{ - ReturnAccessToken: "accessToken", - ReturnRefreshToken: "refreshToken", - } - p := ConfigBackedCachingProvider{ - identifier: "iamidentifier", - config: c, - } - - r, err := p.GetTokens() - - Expect(err).NotTo(HaveOccurred()) - Expect(c.GetTokensCalledIdentifier).To(Equal(p.identifier)) - Expect(r).To(Equal(&TokenResult{ - AccessToken: c.ReturnAccessToken, - RefreshToken: c.ReturnRefreshToken, - })) - }) - - It("caches the tokens in the config provider", func() { - c := &mockConfigProvider{} - p := ConfigBackedCachingProvider{ - identifier: "iamidentifier", - config: c, - } - toSave := &TokenResult{ - AccessToken: "accessToken", - RefreshToken: "refreshToken", - } - - p.CacheTokens(toSave) - - Expect(c.SavedIdentifier).To(Equal(p.identifier)) - Expect(c.SavedAccessToken).To(Equal(toSave.AccessToken)) - Expect(c.SavedRefreshToken).To(Equal(toSave.RefreshToken)) - }) - }) -}) diff --git a/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider.go b/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider.go deleted file mode 100644 index 32986b7314..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider.go +++ /dev/null @@ -1,58 +0,0 @@ -// 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 oauth2 - -import ( - "encoding/json" - "net/http" - "net/url" - "path" - - "github.com/pkg/errors" -) - -// OIDCWellKnownEndpoints holds the well known OIDC endpoints -type OIDCWellKnownEndpoints struct { - AuthorizationEndpoint string `json:"authorization_endpoint"` - TokenEndpoint string `json:"token_endpoint"` - DeviceAuthorizationEndpoint string `json:"device_authorization_endpoint"` -} - -// GetOIDCWellKnownEndpointsFromIssuerURL gets the well known endpoints for the -// passed in issuer url -func GetOIDCWellKnownEndpointsFromIssuerURL(issuerURL string) (*OIDCWellKnownEndpoints, error) { - u, err := url.Parse(issuerURL) - if err != nil { - return nil, errors.Wrap(err, "could not parse issuer url to build well known endpoints") - } - u.Path = path.Join(u.Path, ".well-known/openid-configuration") - - r, err := http.Get(u.String()) - if err != nil { - return nil, errors.Wrapf(err, "could not get well known endpoints from url %s", u.String()) - } - defer r.Body.Close() - - var wkEndpoints OIDCWellKnownEndpoints - err = json.NewDecoder(r.Body).Decode(&wkEndpoints) - if err != nil { - return nil, errors.Wrap(err, "could not decode json body when getting well known endpoints") - } - - return &wkEndpoints, nil -} diff --git a/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider_test.go b/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider_test.go deleted file mode 100644 index 4ebce3b6ca..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/oidc_endpoint_provider_test.go +++ /dev/null @@ -1,92 +0,0 @@ -// 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 oauth2 - -import ( - "encoding/json" - "net/http" - "net/http/httptest" - - . "github.com/onsi/ginkgo" - . "github.com/onsi/gomega" -) - -var _ = Describe("GetOIDCWellKnownEndpointsFromIssuerURL", func() { - It("calls and gets the well known data from the correct endpoint for the issuer", func() { - var req *http.Request - wkEndpointsResp := OIDCWellKnownEndpoints{ - AuthorizationEndpoint: "the-auth-endpoint", TokenEndpoint: "the-token-endpoint"} - responseBytes, err := json.Marshal(wkEndpointsResp) - Expect(err).ToNot(HaveOccurred()) - - ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - req = r - - w.WriteHeader(http.StatusOK) - w.Write(responseBytes) - - })) - defer ts.Close() - - endpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL(ts.URL) - - Expect(err).ToNot(HaveOccurred()) - Expect(*endpoints).To(Equal(wkEndpointsResp)) - Expect(req.URL.Path).To(Equal("/.well-known/openid-configuration")) - }) - - It("errors when url.Parse errors", func() { - endpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL("://") - - Expect(err).To(HaveOccurred()) - Expect(err.Error()).To(Equal( - "could not parse issuer url to build well known endpoints: parse ://: missing protocol scheme")) - Expect(endpoints).To(BeNil()) - }) - - It("errors when the get errors", func() { - endpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL("https://") - - Expect(err).To(HaveOccurred()) - Expect(err.Error()).To(Equal( - "could not get well known endpoints from url https://.well-known/openid-configuration: " + - "Get https://.well-known/openid-configuration: dial tcp: lookup .well-known: no such host")) - Expect(endpoints).To(BeNil()) - }) - - It("errors when the json decoder errors", func() { - var req *http.Request - - ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - req = r - - w.WriteHeader(http.StatusOK) - w.Write([]byte("<")) - - })) - defer ts.Close() - - endpoints, err := GetOIDCWellKnownEndpointsFromIssuerURL(ts.URL) - - Expect(err).To(HaveOccurred()) - Expect(err.Error()).To(Equal("could not decode json body when getting well" + - " known endpoints: invalid character '<' looking for beginning of value")) - Expect(endpoints).To(BeNil()) - Expect(req.URL.Path).To(Equal("/.well-known/openid-configuration")) - }) -}) diff --git a/pulsaradmin/pkg/auth/oauth2/plugin/clock.go b/pulsaradmin/pkg/auth/oauth2/plugin/clock.go deleted file mode 100644 index b2cf4a985c..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/plugin/clock.go +++ /dev/null @@ -1,99 +0,0 @@ -// 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 plugin - -import "time" - -// Clock allows for injecting fake or real clocks into code that -// needs to do arbitrary things based on time. -type Clock interface { - Now() time.Time - Since(time.Time) time.Duration - After(d time.Duration) <-chan time.Time - NewTimer(d time.Duration) Timer - Sleep(d time.Duration) - Tick(d time.Duration) <-chan time.Time -} - -var _ = Clock(RealClock{}) - -// RealClock really calls time.Now() -type RealClock struct{} - -// Now returns the current time. -func (RealClock) Now() time.Time { - return time.Now() -} - -// Since returns time since the specified timestamp. -func (RealClock) Since(ts time.Time) time.Duration { - return time.Since(ts) -} - -// After is the same as time.After(d). -func (RealClock) After(d time.Duration) <-chan time.Time { - return time.After(d) -} - -// NewTimer is the same as time.NewTimer(d) -func (RealClock) NewTimer(d time.Duration) Timer { - return &realTimer{ - timer: time.NewTimer(d), - } -} - -// Tick is the same as time.Tick(d) -// nolint -func (RealClock) Tick(d time.Duration) <-chan time.Time { - return time.Tick(d) -} - -// Sleep is the same as time.Sleep(d) -func (RealClock) Sleep(d time.Duration) { - time.Sleep(d) -} - -// Timer allows for injecting fake or real timers into code that -// needs to do arbitrary things based on time. -type Timer interface { - C() <-chan time.Time - Stop() bool - Reset(d time.Duration) bool -} - -var _ = Timer(&realTimer{}) - -// realTimer is backed by an actual time.Timer. -type realTimer struct { - timer *time.Timer -} - -// C returns the underlying timer's channel. -func (r *realTimer) C() <-chan time.Time { - return r.timer.C -} - -// Stop calls Stop() on the underlying timer. -func (r *realTimer) Stop() bool { - return r.timer.Stop() -} - -// Reset calls Reset() on the underlying timer. -func (r *realTimer) Reset(d time.Duration) bool { - return r.timer.Reset(d) -} diff --git a/pulsaradmin/pkg/auth/oauth2/plugin/fake_clock.go b/pulsaradmin/pkg/auth/oauth2/plugin/fake_clock.go deleted file mode 100644 index 099e0793d2..0000000000 --- a/pulsaradmin/pkg/auth/oauth2/plugin/fake_clock.go +++ /dev/null @@ -1,270 +0,0 @@ -// 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 plugin - -import ( - "sync" - "time" -) - -var ( - _ = Clock(&FakeClock{}) - _ = Clock(&IntervalClock{}) -) - -// FakeClock implements Clock, but returns an arbitrary time. -type FakeClock struct { - lock sync.RWMutex - time time.Time - - // waiters are waiting for the fake time to pass their specified time - waiters []*fakeClockWaiter -} - -type fakeClockWaiter struct { - targetTime time.Time - stepInterval time.Duration - skipIfBlocked bool - destChan chan time.Time - fired bool -} - -// NewFakeClock constructs a fake clock set to the provided time. -func NewFakeClock(t time.Time) *FakeClock { - return &FakeClock{ - time: t, - } -} - -// Now returns f's time. -func (f *FakeClock) Now() time.Time { - f.lock.RLock() - defer f.lock.RUnlock() - return f.time -} - -// Since returns time since the time in f. -func (f *FakeClock) Since(ts time.Time) time.Duration { - f.lock.RLock() - defer f.lock.RUnlock() - return f.time.Sub(ts) -} - -// After is the fake version of time.After(d). -func (f *FakeClock) After(d time.Duration) <-chan time.Time { - f.lock.Lock() - defer f.lock.Unlock() - stopTime := f.time.Add(d) - ch := make(chan time.Time, 1) // Don't block! - f.waiters = append(f.waiters, &fakeClockWaiter{ - targetTime: stopTime, - destChan: ch, - }) - return ch -} - -// NewTimer constructs a fake timer, akin to time.NewTimer(d). -func (f *FakeClock) NewTimer(d time.Duration) Timer { - f.lock.Lock() - defer f.lock.Unlock() - stopTime := f.time.Add(d) - ch := make(chan time.Time, 1) // Don't block! - timer := &fakeTimer{ - fakeClock: f, - waiter: fakeClockWaiter{ - targetTime: stopTime, - destChan: ch, - }, - } - f.waiters = append(f.waiters, &timer.waiter) - return timer -} - -// Tick constructs a fake ticker, akin to time.Tick -func (f *FakeClock) Tick(d time.Duration) <-chan time.Time { - if d <= 0 { - return nil - } - f.lock.Lock() - defer f.lock.Unlock() - tickTime := f.time.Add(d) - ch := make(chan time.Time, 1) // hold one tick - f.waiters = append(f.waiters, &fakeClockWaiter{ - targetTime: tickTime, - stepInterval: d, - skipIfBlocked: true, - destChan: ch, - }) - - return ch -} - -// Step moves the clock by Duration and notifies anyone that's called After, -// Tick, or NewTimer. -func (f *FakeClock) Step(d time.Duration) { - f.lock.Lock() - defer f.lock.Unlock() - f.setTimeLocked(f.time.Add(d)) -} - -// SetTime sets the time. -func (f *FakeClock) SetTime(t time.Time) { - f.lock.Lock() - defer f.lock.Unlock() - f.setTimeLocked(t) -} - -// Actually changes the time and checks any waiters. f must be write-locked. -func (f *FakeClock) setTimeLocked(t time.Time) { - f.time = t - newWaiters := make([]*fakeClockWaiter, 0, len(f.waiters)) - for i := range f.waiters { - w := f.waiters[i] - if !w.targetTime.After(t) { - - if w.skipIfBlocked { - select { - case w.destChan <- t: - w.fired = true - default: - } - } else { - w.destChan <- t - w.fired = true - } - - if w.stepInterval > 0 { - for !w.targetTime.After(t) { - w.targetTime = w.targetTime.Add(w.stepInterval) - } - newWaiters = append(newWaiters, w) - } - - } else { - newWaiters = append(newWaiters, f.waiters[i]) - } - } - f.waiters = newWaiters -} - -// HasWaiters returns true if After has been called on f but not yet satisfied (so you can -// write race-free tests). -func (f *FakeClock) HasWaiters() bool { - f.lock.RLock() - defer f.lock.RUnlock() - return len(f.waiters) > 0 -} - -// Sleep is akin to time.Sleep -func (f *FakeClock) Sleep(d time.Duration) { - f.Step(d) -} - -// IntervalClock implements Clock, but each invocation of Now steps the clock forward the specified duration -type IntervalClock struct { - Time time.Time - Duration time.Duration -} - -// Now returns i's time. -func (i *IntervalClock) Now() time.Time { - i.Time = i.Time.Add(i.Duration) - return i.Time -} - -// Since returns time since the time in i. -func (i *IntervalClock) Since(ts time.Time) time.Duration { - return i.Time.Sub(ts) -} - -// After is unimplemented, will panic. -func (*IntervalClock) After(d time.Duration) <-chan time.Time { - panic("IntervalClock doesn't implement After") -} - -// NewTimer is unimplemented, will panic. -func (*IntervalClock) NewTimer(d time.Duration) Timer { - panic("IntervalClock doesn't implement NewTimer") -} - -// Tick is unimplemented, will panic. -func (*IntervalClock) Tick(d time.Duration) <-chan time.Time { - panic("IntervalClock doesn't implement Tick") -} - -// Sleep is unimplemented, will panic. -func (*IntervalClock) Sleep(d time.Duration) { - panic("IntervalClock doesn't implement Sleep") -} - -var _ = Timer(&fakeTimer{}) - -// fakeTimer implements Timer based on a FakeClock. -type fakeTimer struct { - fakeClock *FakeClock - waiter fakeClockWaiter -} - -// C returns the channel that notifies when this timer has fired. -func (f *fakeTimer) C() <-chan time.Time { - return f.waiter.destChan -} - -// Stop stops the timer and returns true if the timer has not yet fired, or false otherwise. -func (f *fakeTimer) Stop() bool { - f.fakeClock.lock.Lock() - defer f.fakeClock.lock.Unlock() - - newWaiters := make([]*fakeClockWaiter, 0, len(f.fakeClock.waiters)) - for i := range f.fakeClock.waiters { - w := f.fakeClock.waiters[i] - if w != &f.waiter { - newWaiters = append(newWaiters, w) - } - } - - f.fakeClock.waiters = newWaiters - - return !f.waiter.fired -} - -// Reset resets the timer to the fake clock's "now" + d. It returns true if the timer has not yet -// fired, or false otherwise. -func (f *fakeTimer) Reset(d time.Duration) bool { - f.fakeClock.lock.Lock() - defer f.fakeClock.lock.Unlock() - - active := !f.waiter.fired - - f.waiter.fired = false - f.waiter.targetTime = f.fakeClock.time.Add(d) - - var isWaiting bool - for i := range f.fakeClock.waiters { - w := f.fakeClock.waiters[i] - if w == &f.waiter { - isWaiting = true - break - } - } - if !isWaiting { - f.fakeClock.waiters = append(f.fakeClock.waiters, &f.waiter) - } - - return active -} diff --git a/pulsaradmin/pkg/auth/oauth2_test.go b/pulsaradmin/pkg/auth/oauth2_test.go index 7efb7af81d..5d412dd3fb 100644 --- a/pulsaradmin/pkg/auth/oauth2_test.go +++ b/pulsaradmin/pkg/auth/oauth2_test.go @@ -25,6 +25,8 @@ import ( "os" "testing" + "github.com/apache/pulsar-client-go/oauth2" + "github.com/apache/pulsar-client-go/oauth2/store" "github.com/pkg/errors" "github.com/stretchr/testify/assert" ) @@ -59,7 +61,7 @@ func mockOAuthServer() *httptest.Server { } // mockKeyFile will mock a temp key file for testing. -func mockKeyFile() (string, error) { +func mockKeyFile(server string) (string, error) { pwd, err := os.Getwd() if err != nil { return "", err @@ -68,13 +70,14 @@ func mockKeyFile() (string, error) { if err != nil { return "", err } - _, err = kf.WriteString(`{ + _, err = kf.WriteString(fmt.Sprintf(`{ "type":"sn_service_account", "client_id":"client-id", "client_secret":"client-secret", "client_email":"oauth@test.org", - "issuer_url":"http://issue-url" -}`) + "issuer_url":"%s" +}`, server)) + if err != nil { return "", err } @@ -85,22 +88,49 @@ func mockKeyFile() (string, error) { func TestOauth2(t *testing.T) { server := mockOAuthServer() defer server.Close() - kf, err := mockKeyFile() + kf, err := mockKeyFile(server.URL) defer os.Remove(kf) if err != nil { t.Fatal(errors.Wrap(err, "create mocked key file failed")) } - transport := http.DefaultTransport.(*http.Transport) + issuer := oauth2.Issuer{ + IssuerEndpoint: server.URL, + ClientID: "client-id", + Audience: server.URL, + } + + memoryStore := store.NewMemoryStore() + err = saveGrant(memoryStore, kf, issuer.Audience) + if err != nil { + t.Fatal(err) + } - auth, err := NewAuthenticationOAuth2(server.URL, "client-id", "audience", kf, transport) + auth, err := NewAuthenticationOauth2(issuer, memoryStore) if err != nil { t.Fatal(err) } - token, err := auth.getToken(auth.issuer) + token, err := auth.source.Token() if err != nil { t.Fatal(err) } - assert.Equal(t, "token-content", token) + assert.Equal(t, "token-content", token.AccessToken) +} + +func saveGrant(store store.Store, keyFile, audience string) error { + flow, err := oauth2.NewDefaultClientCredentialsFlow(oauth2.ClientCredentialsFlowOptions{ + KeyFile: keyFile, + AdditionalScopes: nil, + }) + if err != nil { + return err + } + + grant, err := flow.Authorize(audience) + if err != nil { + return err + } + + return store.SaveGrant(audience, *grant) } diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index e2b75509f0..856b34bf89 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -83,6 +83,21 @@ func New(config *common.Config) (Client, error) { return c, err } +func NewWithAuthProvider(config *common.Config, auth auth.Provider) Client { + c := &pulsarClient{ + APIVersion: config.PulsarAPIVersion, + Client: &cli.Client{ + ServiceURL: config.WebServiceURL, + VersionInfo: ReleaseVersion, + HTTPClient: &http.Client{ + Timeout: DefaultHTTPTimeOutDuration, + Transport: auth, + }, + }, + } + return c +} + func (c *pulsarClient) endpoint(componentPath string, parts ...string) string { return path.Join(utils.MakeHTTPPath(c.APIVersion.String(), componentPath), path.Join(parts...)) } diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/pulsar/subscription.go index 9852aa9f97..f4c41e00ea 100644 --- a/pulsaradmin/pkg/pulsar/subscription.go +++ b/pulsaradmin/pkg/pulsar/subscription.go @@ -27,6 +27,7 @@ import ( "strconv" "strings" + // nolint "github.com/golang/protobuf/proto" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) diff --git a/pulsaradmin/pkg/pulsar/utils/message.go b/pulsaradmin/pkg/pulsar/utils/message.go index d6207a2c1a..f60bbff66a 100644 --- a/pulsaradmin/pkg/pulsar/utils/message.go +++ b/pulsaradmin/pkg/pulsar/utils/message.go @@ -17,6 +17,7 @@ package utils +// nolint import "github.com/golang/protobuf/proto" type Message struct { From bdaa430dac7e76ed8c1be4836cf87a9169fab269 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 27 Jul 2020 16:02:12 +0800 Subject: [PATCH 170/348] Fix the TLS not working when enable the oauth2 authentication (streamnative/pulsarctl#242) * Fix the TLS not working when enable the oauth2 authenticatioin --- Fixes streamnative/pulsarctl#240 *Motivation* The pulsarctl client has a default TLS config initialize before initiating the auth provider. We need to make sure the oauth2 provider has the config as well. --- pulsaradmin/pkg/auth/auth_provider.go | 7 +-- pulsaradmin/pkg/auth/oauth2.go | 73 ++++++++++++++++++++------- pulsaradmin/pkg/auth/oauth2_test.go | 2 +- pulsaradmin/pkg/auth/tls.go | 4 ++ pulsaradmin/pkg/auth/token.go | 4 ++ pulsaradmin/pkg/pulsar/admin.go | 7 ++- 6 files changed, 73 insertions(+), 24 deletions(-) diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index ade68bc1e3..edac0edddc 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -32,6 +32,7 @@ import ( type Provider interface { RoundTrip(req *http.Request) (*http.Response, error) Transport() http.RoundTripper + WithTransport(tripper http.RoundTripper) } type Transport struct { @@ -40,7 +41,7 @@ type Transport struct { func GetAuthProvider(config *common.Config) (*Provider, error) { var provider Provider - defaultTransport := getDefaultTransport(config) + defaultTransport := GetDefaultTransport(config) var err error switch config.AuthPlugin { case TLSPluginName: @@ -59,13 +60,13 @@ func GetAuthProvider(config *common.Config) (*Provider, error) { provider, err = NewAuthenticationOAuth2WithParams( config.IssuerEndpoint, config.ClientID, - config.Audience) + config.Audience, defaultTransport) } } return &provider, err } -func getDefaultTransport(config *common.Config) http.RoundTripper { +func GetDefaultTransport(config *common.Config) http.RoundTripper { transport := http.DefaultTransport.(*http.Transport) tlsConfig := &tls.Config{ InsecureSkipVerify: config.TLSAllowInsecureConnection, diff --git a/pulsaradmin/pkg/auth/oauth2.go b/pulsaradmin/pkg/auth/oauth2.go index 00a50b745a..a15a555322 100644 --- a/pulsaradmin/pkg/auth/oauth2.go +++ b/pulsaradmin/pkg/auth/oauth2.go @@ -18,7 +18,6 @@ package auth import ( - "fmt" "net/http" "path/filepath" @@ -37,14 +36,15 @@ const ( ) type OAuth2Provider struct { - clock clock2.RealClock - issuer oauth2.Issuer - store store.Store - source cache.CachingTokenSource - T http.RoundTripper + clock clock2.RealClock + issuer oauth2.Issuer + store store.Store + source cache.CachingTokenSource + defaultTransport http.RoundTripper + tokenTransport *transport } -func NewAuthenticationOauth2(issuer oauth2.Issuer, store store.Store) (*OAuth2Provider, error) { +func NewAuthenticationOAuth2(issuer oauth2.Issuer, store store.Store) (*OAuth2Provider, error) { p := &OAuth2Provider{ clock: clock2.RealClock{}, issuer: issuer, @@ -59,10 +59,40 @@ func NewAuthenticationOauth2(issuer oauth2.Issuer, store store.Store) (*OAuth2Pr return p, nil } +// NewAuthenticationOAuth2WithDefaultFlow uses memory to save the grant +func NewAuthenticationOAuth2WithDefaultFlow(issuer oauth2.Issuer, keyFile string) (Provider, error) { + st := store.NewMemoryStore() + flow, err := oauth2.NewDefaultClientCredentialsFlow(oauth2.ClientCredentialsFlowOptions{ + KeyFile: keyFile, + }) + if err != nil { + return nil, err + } + + grant, err := flow.Authorize(issuer.Audience) + if err != nil { + return nil, err + } + + err = st.SaveGrant(issuer.Audience, *grant) + if err != nil { + return nil, err + } + + p := &OAuth2Provider{ + clock: clock2.RealClock{}, + issuer: issuer, + store: st, + } + + return p, p.loadGrant() +} + func NewAuthenticationOAuth2WithParams( issueEndpoint, clientID, - audience string) (*OAuth2Provider, error) { + audience string, + transport http.RoundTripper) (*OAuth2Provider, error) { issuer := oauth2.Issuer{ IssuerEndpoint: issueEndpoint, @@ -76,9 +106,10 @@ func NewAuthenticationOAuth2WithParams( } p := &OAuth2Provider{ - clock: clock2.RealClock{}, - issuer: issuer, - store: keyringStore, + clock: clock2.RealClock{}, + issuer: issuer, + store: keyringStore, + defaultTransport: transport, } err = p.loadGrant() @@ -108,11 +139,22 @@ func (o *OAuth2Provider) initCache(grant *oauth2.AuthorizationGrant) error { return err } o.source = source + o.tokenTransport = &transport{ + source: o.source, + wrapped: &xoauth2.Transport{ + Source: o.source, + Base: o.defaultTransport, + }, + } return nil } func (o *OAuth2Provider) RoundTrip(req *http.Request) (*http.Response, error) { - return o.Transport().RoundTrip(req) + return o.tokenTransport.RoundTrip(req) +} + +func (o *OAuth2Provider) WithTransport(tripper http.RoundTripper) { + o.defaultTransport = tripper } func (o *OAuth2Provider) Transport() http.RoundTripper { @@ -120,7 +162,7 @@ func (o *OAuth2Provider) Transport() http.RoundTripper { source: o.source, wrapped: &xoauth2.Transport{ Source: o.source, - Base: o.T, + Base: o.defaultTransport, }, } } @@ -146,11 +188,6 @@ func (t *transport) RoundTrip(req *http.Request) (*http.Response, error) { return t.wrapped.Base.RoundTrip(req) } - token, err := t.source.Token() - if err != nil { - fmt.Println(err.Error()) - } - fmt.Println(token.AccessToken) res, err := t.wrapped.RoundTrip(req) if err != nil { return nil, err diff --git a/pulsaradmin/pkg/auth/oauth2_test.go b/pulsaradmin/pkg/auth/oauth2_test.go index 5d412dd3fb..f832af58cc 100644 --- a/pulsaradmin/pkg/auth/oauth2_test.go +++ b/pulsaradmin/pkg/auth/oauth2_test.go @@ -106,7 +106,7 @@ func TestOauth2(t *testing.T) { t.Fatal(err) } - auth, err := NewAuthenticationOauth2(issuer, memoryStore) + auth, err := NewAuthenticationOAuth2(issuer, memoryStore) if err != nil { t.Fatal(err) } diff --git a/pulsaradmin/pkg/auth/tls.go b/pulsaradmin/pkg/auth/tls.go index 2cad869d00..3ac2e303f0 100644 --- a/pulsaradmin/pkg/auth/tls.go +++ b/pulsaradmin/pkg/auth/tls.go @@ -86,3 +86,7 @@ func (p *TLSAuthProvider) configTLS() error { transport.TLSClientConfig.Certificates = []tls.Certificate{*cert} return nil } + +func (p *TLSAuthProvider) WithTransport(tripper http.RoundTripper) { + p.T = tripper +} diff --git a/pulsaradmin/pkg/auth/token.go b/pulsaradmin/pkg/auth/token.go index b825f669f0..18088a5427 100644 --- a/pulsaradmin/pkg/auth/token.go +++ b/pulsaradmin/pkg/auth/token.go @@ -78,3 +78,7 @@ func (p *TokenAuthProvider) RoundTrip(req *http.Request) (*http.Response, error) func (p *TokenAuthProvider) Transport() http.RoundTripper { return p.T } + +func (p *TokenAuthProvider) WithTransport(tripper http.RoundTripper) { + p.T = tripper +} diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 856b34bf89..560a000034 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -83,7 +83,10 @@ func New(config *common.Config) (Client, error) { return c, err } -func NewWithAuthProvider(config *common.Config, auth auth.Provider) Client { +func NewWithAuthProvider(config *common.Config, authProvider auth.Provider) Client { + defaultTransport := auth.GetDefaultTransport(config) + authProvider.WithTransport(defaultTransport) + c := &pulsarClient{ APIVersion: config.PulsarAPIVersion, Client: &cli.Client{ @@ -91,7 +94,7 @@ func NewWithAuthProvider(config *common.Config, auth auth.Provider) Client { VersionInfo: ReleaseVersion, HTTPClient: &http.Client{ Timeout: DefaultHTTPTimeOutDuration, - Transport: auth, + Transport: authProvider, }, }, } From c6f7ddff824ca9274b234ddef95ef93101a58d70 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Milo=C5=A1=20Matija=C5=A1evi=C4=87?= Date: Mon, 23 Nov 2020 10:47:30 +0100 Subject: [PATCH 171/348] Fixes default api version to be v2 (streamnative/pulsarctl#250) * default undefined api version Signed-off-by: milos-matijasevic * add test * use default variable instead of string literal * case undefined return default api version Signed-off-by: milos-matijasevic --- pulsaradmin/pkg/pulsar/common/api_version.go | 5 ++++- pulsaradmin/pkg/pulsar/common/api_version_test.go | 2 ++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/common/api_version.go b/pulsaradmin/pkg/pulsar/common/api_version.go index f490828daf..6001c12f06 100644 --- a/pulsaradmin/pkg/pulsar/common/api_version.go +++ b/pulsaradmin/pkg/pulsar/common/api_version.go @@ -20,7 +20,8 @@ package common type APIVersion int const ( - V1 APIVersion = iota + undefined APIVersion = iota + V1 V2 V3 ) @@ -29,6 +30,8 @@ const DefaultAPIVersion = "v2" func (v APIVersion) String() string { switch v { + case undefined: + return DefaultAPIVersion case V1: return "" case V2: diff --git a/pulsaradmin/pkg/pulsar/common/api_version_test.go b/pulsaradmin/pkg/pulsar/common/api_version_test.go index 5281922dfb..335b774136 100644 --- a/pulsaradmin/pkg/pulsar/common/api_version_test.go +++ b/pulsaradmin/pkg/pulsar/common/api_version_test.go @@ -27,4 +27,6 @@ func TestApiVersion_String(t *testing.T) { assert.Equal(t, "", V1.String()) assert.Equal(t, "v2", V2.String()) assert.Equal(t, "v3", V3.String()) + var undefinedAPIVersion APIVersion + assert.Equal(t, DefaultAPIVersion, undefinedAPIVersion.String()) } From 806a394d00e6f84fc7eb1e45976e9269649e3a53 Mon Sep 17 00:00:00 2001 From: Evan Leif Battaglia Date: Mon, 23 Nov 2020 17:33:27 -0700 Subject: [PATCH 172/348] Escape special characters in parts of the URL (streamnative/pulsarctl#252) Our subscripton name has a slash in it, and "pulsarctl subscriptions seek" (resetcursor) wasn't working. Looking at our Pulsar Admin HTTP logs and the pulsar-admin command's source code, it appears they actually URL-encode all path components (and as a result, the subscription apparently gets double-encoded). The URL in our logs looked something like: POST /admin/v2/persistent/mytenant/mynamespace/mytopic/subscription/my%252Fsub/resetcursor/1605911005739 Also, clearly we should be using PathEscape rather than QueryEscape, since we are using path components. QueryEscape will turn a space into "+" instead of "%20". You can also validate that this is the right thing to go by running the following commands with both pulsarctl and pulsar-admin pulsarctl functions stats --fqfn "tenant/namespace/foo?bar" -> currently in pulsarctl this shows an error "Function foo doesn't exist". With this fix, and with pulsar-admin, it shows "Function foo?bar doesn't exist" pulsarctl functions stats --fqfn "tenant/namespace/foo bar" -> with this fix pulsarctl shows "Function foo bar doesn't exist", which is what pulsar-admin also shows. (If I were to use QueryEscape, it would show "Function foo+bar doesn't exist") See the pulsar-admin code: pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java * in resetCursorAsync(...): calls Codec.encode which uses URLEncoder to encode sub name * in topicPath: calls WebTarget.addParts(...), which also uses URLEncoder to encode each of the parts Co-authored-by: Evan Battaglia --- pulsaradmin/pkg/pulsar/admin.go | 10 +++++++- pulsaradmin/pkg/pulsar/admin_test.go | 33 ++++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/subscription.go | 16 ++++++------- 3 files changed, 50 insertions(+), 9 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/admin_test.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 560a000034..3f61af2f65 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -20,6 +20,7 @@ package pulsar import ( "fmt" "net/http" + "net/url" "path" "github.com/streamnative/pulsar-admin-go/pkg/auth" @@ -102,5 +103,12 @@ func NewWithAuthProvider(config *common.Config, authProvider auth.Provider) Clie } func (c *pulsarClient) endpoint(componentPath string, parts ...string) string { - return path.Join(utils.MakeHTTPPath(c.APIVersion.String(), componentPath), path.Join(parts...)) + escapedParts := make([]string, len(parts)) + for i, part := range parts { + escapedParts[i] = url.PathEscape(part) + } + return path.Join( + utils.MakeHTTPPath(c.APIVersion.String(), componentPath), + path.Join(escapedParts...), + ) } diff --git a/pulsaradmin/pkg/pulsar/admin_test.go b/pulsaradmin/pkg/pulsar/admin_test.go new file mode 100644 index 0000000000..91268d9846 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/admin_test.go @@ -0,0 +1,33 @@ +// 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 pulsar + +import ( + "testing" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" + + "github.com/stretchr/testify/assert" +) + +func TestPulsarClientEndpointEscapes(t *testing.T) { + client := pulsarClient{Client: nil, APIVersion: common.V2} + actual := client.endpoint("/myendpoint", "abc%? /def", "ghi") + expected := "/admin/v2/myendpoint/abc%25%3F%20%2Fdef/ghi" + assert.Equal(t, expected, actual) +} diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/pulsar/subscription.go index f4c41e00ea..bffffc181e 100644 --- a/pulsaradmin/pkg/pulsar/subscription.go +++ b/pulsaradmin/pkg/pulsar/subscription.go @@ -87,12 +87,12 @@ func (c *pulsarClient) Subscriptions() Subscriptions { } func (s *subscriptions) Create(topic utils.TopicName, sName string, messageID utils.MessageID) error { - endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName)) + endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName)) return s.pulsar.Client.Put(endpoint, messageID) } func (s *subscriptions) Delete(topic utils.TopicName, sName string) error { - endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName)) + endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName)) return s.pulsar.Client.Delete(endpoint) } @@ -103,33 +103,33 @@ func (s *subscriptions) List(topic utils.TopicName) ([]string, error) { } func (s *subscriptions) ResetCursorToMessageID(topic utils.TopicName, sName string, id utils.MessageID) error { - endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "resetcursor") + endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName), "resetcursor") return s.pulsar.Client.Post(endpoint, id) } func (s *subscriptions) ResetCursorToTimestamp(topic utils.TopicName, sName string, timestamp int64) error { endpoint := s.pulsar.endpoint( - s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), + s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName), "resetcursor", strconv.FormatInt(timestamp, 10)) return s.pulsar.Client.Post(endpoint, "") } func (s *subscriptions) ClearBacklog(topic utils.TopicName, sName string) error { endpoint := s.pulsar.endpoint( - s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), "skip_all") + s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName), "skip_all") return s.pulsar.Client.Post(endpoint, "") } func (s *subscriptions) SkipMessages(topic utils.TopicName, sName string, n int64) error { endpoint := s.pulsar.endpoint( - s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), + s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName), "skip", strconv.FormatInt(n, 10)) return s.pulsar.Client.Post(endpoint, "") } func (s *subscriptions) ExpireMessages(topic utils.TopicName, sName string, expire int64) error { endpoint := s.pulsar.endpoint( - s.basePath, topic.GetRestPath(), s.SubPath, url.QueryEscape(sName), + s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName), "expireMessages", strconv.FormatInt(expire, 10)) return s.pulsar.Client.Post(endpoint, "") } @@ -159,7 +159,7 @@ func (s *subscriptions) PeekMessages(topic utils.TopicName, sName string, n int) } func (s *subscriptions) peekNthMessage(topic utils.TopicName, sName string, pos int) ([]*utils.Message, error) { - endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), "subscription", url.QueryEscape(sName), + endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), "subscription", url.PathEscape(sName), "position", strconv.Itoa(pos)) resp, err := s.pulsar.Client.MakeRequest(http.MethodGet, endpoint) From 25389b257e9b2935b5f52be5a7aaa06a4be833e6 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 24 Nov 2020 15:59:16 +0800 Subject: [PATCH 173/348] Improve the current unit tests (streamnative/pulsarctl#253) *Motivation* Improve the unit test and wrap it in the docker to run. --- .../pkg/pulsar/common/algorithm/algorithm/algorithm_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go index 50a77423fa..252b80d595 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go @@ -41,8 +41,8 @@ var testData = []struct { } func TestGetSignatureAlgorithm(t *testing.T) { + // test ci status for _, data := range testData { - t.Logf("test case: %+v", data) switch data.name { case "HMAC": testHMAC(t, data.algorithm) From 0f3d1b5d93786a6759a1d9d9bc468d0cf893145a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Milo=C5=A1=20Matija=C5=A1evi=C4=87?= Date: Fri, 11 Dec 2020 01:42:17 +0100 Subject: [PATCH 174/348] Updating fields in utils data (streamnative/pulsarctl#254) * offloaded instead of timestamp in LedgerInfo Signed-off-by: milos-matijasevic * updating example in internal info Signed-off-by: milos-matijasevic Co-authored-by: Yong Zhang --- pulsaradmin/pkg/pulsar/utils/data.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index 2021642d30..f8d6f755b5 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -261,7 +261,7 @@ type LedgerInfo struct { LedgerID int64 `json:"ledgerId"` Entries int64 `json:"entries"` Size int64 `json:"size"` - Timestamp int64 `json:"timestamp"` + Offloaded bool `json:"offloaded"` } type CursorInfo struct { From 2e353e1d5887a55306f79052536b0defd7cfe0e9 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 29 Dec 2020 04:00:28 +0800 Subject: [PATCH 175/348] Improve the current CI checks (streamnative/pulsarctl#258) **Motivation** Improve CI checks. And make the tests can run in docker. --- pulsaradmin/go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index 07ba304916..c13b19732a 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -3,7 +3,7 @@ module github.com/streamnative/pulsar-admin-go go 1.12 require ( - github.com/99designs/keyring v1.1.5 + github.com/99designs/keyring v1.1.6 github.com/apache/pulsar-client-go/oauth2 v0.0.0-20200715083626-b9f8c5cedefb github.com/dgrijalva/jwt-go v3.2.0+incompatible github.com/docker/go-connections v0.4.0 From ad4a9650133daf1a66687661b87ff84e13187c16 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 15 Jan 2021 18:42:58 +0800 Subject: [PATCH 176/348] Handle the null response body (streamnative/pulsarctl#272) * Handle the null response body --- *Motivation* Handle the null response body in the get request. * fix the failed tests --- pulsaradmin/pkg/cli/client.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index 30d10fe89f..569ff7f049 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -120,6 +120,9 @@ func (c *Client) GetWithQueryParams(endpoint string, obj interface{}, params map if obj != nil { if err := decodeJSONBody(resp, &obj); err != nil { + if err == io.EOF { + return nil, nil + } return nil, err } } else if !decode { From 1586533d6dc1306fd710b5fe0b46d7093701751a Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 27 Jan 2021 20:19:59 +0800 Subject: [PATCH 177/348] Fix the function download command issue (streamnative/pulsarctl#275) * Fix the function download command issue --- Fixes streamnative/pulsarctl#270 *Motivation* Fix the function download command doesn't write the response file data into the target file. **Modification** - Fix download command - Add upload command - Add integration test for uploading and download the file and check the file sha256 value --- pulsaradmin/pkg/cli/client.go | 20 ++++++++--- pulsaradmin/pkg/pulsar/functions.go | 56 +++++++++++++++++++++++------ 2 files changed, 62 insertions(+), 14 deletions(-) diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index 569ff7f049..fa6c7ff9e7 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -98,6 +98,11 @@ func (c *Client) Get(endpoint string, obj interface{}) error { func (c *Client) GetWithQueryParams(endpoint string, obj interface{}, params map[string]string, decode bool) ([]byte, error) { + return c.GetWithOptions(endpoint, obj, params, decode, nil) +} + +func (c *Client) GetWithOptions(endpoint string, obj interface{}, params map[string]string, + decode bool, file io.Writer) ([]byte, error) { req, err := c.newRequest(http.MethodGet, endpoint) if err != nil { @@ -126,11 +131,18 @@ func (c *Client) GetWithQueryParams(endpoint string, obj interface{}, params map return nil, err } } else if !decode { - body, err := ioutil.ReadAll(resp.Body) - if err != nil { - return nil, err + if file != nil { + _, err := io.Copy(file, resp.Body) + if err != nil { + return nil, err + } + } else { + body, err := ioutil.ReadAll(resp.Body) + if err != nil { + return nil, err + } + return body, err } - return body, err } return nil, err diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index ee7d168ba4..b9d10c5003 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -128,6 +128,9 @@ type Functions interface { // File: file:/dir/fileName.jar // Http: http://www.repo.com/fileName.jar UpdateFunctionWithURL(functionConfig *utils.FunctionConfig, pkgURL string, updateOptions *utils.UpdateOptions) error + + // Upload function to Pulsar + Upload(sourceFile, path string) error } type functions struct { @@ -282,21 +285,23 @@ func (f *functions) DownloadFunction(path, destinationFile string) error { endpoint := f.pulsar.endpoint(f.basePath, "download") _, err := os.Open(destinationFile) if err != nil { - _, err = os.Create(destinationFile) - if err != nil { - return err + if !os.IsNotExist(err) { + return fmt.Errorf("file %s already exists, please delete "+ + "the file first or change the file name", destinationFile) } } + file, err := os.Create(destinationFile) + if err != nil { + return err + } tmpMap := make(map[string]string) tmpMap["path"] = path - _, err = f.pulsar.Client.GetWithQueryParams(endpoint, nil, tmpMap, false) - + _, err = f.pulsar.Client.GetWithOptions(endpoint, nil, tmpMap, false, file) if err != nil { return err } - return nil } @@ -304,13 +309,17 @@ func (f *functions) DownloadFunctionByNs(destinationFile, tenant, namespace, fun endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, function, "download") _, err := os.Open(destinationFile) if err != nil { - _, err = os.Create(destinationFile) - if err != nil { - return err + if !os.IsNotExist(err) { + return fmt.Errorf("file %s already exists, please delete "+ + "the file first or change the file name", destinationFile) } } + file, err := os.Create(destinationFile) + if err != nil { + return err + } - err = f.pulsar.Client.Get(endpoint, nil) + _, err = f.pulsar.Client.GetWithOptions(endpoint, nil, nil, false, file) if err != nil { return err } @@ -645,3 +654,30 @@ func (f *functions) TriggerFunction(tenant, namespace, name, topic, triggerValue return str, nil } + +func (f *functions) Upload(sourceFile, path string) error { + if strings.TrimSpace(sourceFile) == "" && strings.TrimSpace(path) == "" { + return fmt.Errorf("source file or path is empty") + } + file, err := os.Open(sourceFile) + if err != nil { + return err + } + endpoint := f.pulsar.endpoint(f.basePath, "upload") + var b bytes.Buffer + w := multipart.NewWriter(&b) + writer, err := w.CreateFormFile("data", file.Name()) + if err != nil { + return err + } + _, err = io.Copy(writer, file) + if err != nil { + return err + } + w.WriteField("path", path) + err = w.Close() + if err != nil { + return err + } + return f.pulsar.Client.PostWithMultiPart(endpoint, nil, &b, w.FormDataContentType()) +} From 28a98b95ffa0dec57e096a0a6af81a7f77947d31 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Sat, 20 Feb 2021 17:08:44 +0800 Subject: [PATCH 178/348] Upgrade dependencies (streamnative/pulsarctl#285) --- pulsaradmin/go.mod | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index c13b19732a..ca119dd8bf 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -4,12 +4,12 @@ go 1.12 require ( github.com/99designs/keyring v1.1.6 - github.com/apache/pulsar-client-go/oauth2 v0.0.0-20200715083626-b9f8c5cedefb + github.com/apache/pulsar-client-go/oauth2 v0.0.0-20210220083636-af91e9ca0ee2 github.com/dgrijalva/jwt-go v3.2.0+incompatible github.com/docker/go-connections v0.4.0 github.com/fatih/color v1.7.0 github.com/ghodss/yaml v1.0.0 - github.com/golang/protobuf v1.4.2 + github.com/golang/protobuf v1.4.3 github.com/imdario/mergo v0.3.8 github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b @@ -22,9 +22,9 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.4.0 github.com/testcontainers/testcontainers-go v0.0.10 - golang.org/x/net v0.0.0-20200707034311-ab3426394381 // indirect - golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d - google.golang.org/appengine v1.6.6 // indirect + golang.org/x/net v0.0.0-20210220033124-5f55cee0dc0d // indirect + golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93 + google.golang.org/appengine v1.6.7 // indirect google.golang.org/protobuf v1.25.0 // indirect gopkg.in/yaml.v2 v2.3.0 ) From 8c409bcef767ee4ea89c9f3e30c6214a37b3ac9e Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 19 Mar 2021 15:05:51 +0800 Subject: [PATCH 179/348] Fix the error introduce from the latest pulsar (streamnative/pulsarctl#284) * Fix the error introduce from the latest pulsar --- * Update test image --- pulsaradmin/pkg/cli/client.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index fa6c7ff9e7..465027b24e 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -346,6 +346,9 @@ func encodeJSONBody(obj interface{}) (io.Reader, error) { // decodeJSONBody is used to JSON decode a body func decodeJSONBody(resp *http.Response, out interface{}) error { + if resp.ContentLength == 0 { + return nil + } dec := json.NewDecoder(resp.Body) return dec.Decode(out) } From dea508e3ed421a03b137fcb427533e0ee8e16546 Mon Sep 17 00:00:00 2001 From: Peter Tinti Date: Fri, 14 May 2021 00:05:59 -0400 Subject: [PATCH 180/348] Adds namespace topic auto-creation configuration (streamnative/pulsarctl#297) * adds namespace topic auto-creation config commands * makes namespace Policies aware of autoTopicCreation --- pulsaradmin/pkg/pulsar/namespace.go | 16 +++++++ pulsaradmin/pkg/pulsar/utils/policies.go | 1 + .../utils/topic_auto_creation_config.go | 24 +++++++++++ pulsaradmin/pkg/pulsar/utils/topic_type.go | 43 +++++++++++++++++++ 4 files changed, 84 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/utils/topic_auto_creation_config.go create mode 100644 pulsaradmin/pkg/pulsar/utils/topic_type.go diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index 903617f869..4c8f8600df 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -76,6 +76,12 @@ type Namespaces interface { // RemoveBacklogQuota removes a backlog quota policy from a namespace RemoveBacklogQuota(namespace string) error + // SetTopicAutoCreation sets topic auto-creation config for a namespace, overriding broker settings + SetTopicAutoCreation(namespace utils.NameSpaceName, config utils.TopicAutoCreationConfig) error + + // RemoveTopicAutoCreation removes topic auto-creation config for a namespace, defaulting to broker settings + RemoveTopicAutoCreation(namespace utils.NameSpaceName) error + // SetSchemaValidationEnforced sets schema validation enforced for namespace SetSchemaValidationEnforced(namespace utils.NameSpaceName, schemaValidationEnforced bool) error @@ -416,6 +422,16 @@ func (n *namespaces) RemoveBacklogQuota(namespace string) error { return n.pulsar.Client.DeleteWithQueryParams(endpoint, params) } +func (n *namespaces) SetTopicAutoCreation(namespace utils.NameSpaceName, config utils.TopicAutoCreationConfig) error { + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "autoTopicCreation") + return n.pulsar.Client.Post(endpoint, &config) +} + +func (n *namespaces) RemoveTopicAutoCreation(namespace utils.NameSpaceName) error { + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "autoTopicCreation") + return n.pulsar.Client.Delete(endpoint) +} + func (n *namespaces) SetSchemaValidationEnforced(namespace utils.NameSpaceName, schemaValidationEnforced bool) error { endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "schemaValidationEnforced") return n.pulsar.Client.Post(endpoint, schemaValidationEnforced) diff --git a/pulsaradmin/pkg/pulsar/utils/policies.go b/pulsaradmin/pkg/pulsar/utils/policies.go index 3f6f897821..0a93afe208 100644 --- a/pulsaradmin/pkg/pulsar/utils/policies.go +++ b/pulsaradmin/pkg/pulsar/utils/policies.go @@ -49,6 +49,7 @@ type Policies struct { SubscriptionDispatchRate map[string]DispatchRate `json:"subscriptionDispatchRate"` ReplicatorDispatchRate map[string]DispatchRate `json:"replicatorDispatchRate"` ClusterSubscribeRate map[string]SubscribeRate `json:"clusterSubscribeRate"` + TopicAutoCreationConfig TopicAutoCreationConfig `json:"autoTopicCreationOverride"` SchemaCompatibilityStrategy SchemaCompatibilityStrategy `json:"schema_auto_update_compatibility_strategy"` AuthPolicies common.AuthPolicies `json:"auth_policies"` SubscriptionAuthMode SubscriptionAuthMode `json:"subscription_auth_mode"` diff --git a/pulsaradmin/pkg/pulsar/utils/topic_auto_creation_config.go b/pulsaradmin/pkg/pulsar/utils/topic_auto_creation_config.go new file mode 100644 index 0000000000..6664655974 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/utils/topic_auto_creation_config.go @@ -0,0 +1,24 @@ +// 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 utils + +type TopicAutoCreationConfig struct { + Allow bool `json:"allowAutoTopicCreation"` + Type TopicType `json:"topicType"` + Partitions int `json:"defaultNumPartitions"` +} diff --git a/pulsaradmin/pkg/pulsar/utils/topic_type.go b/pulsaradmin/pkg/pulsar/utils/topic_type.go new file mode 100644 index 0000000000..18320594ac --- /dev/null +++ b/pulsaradmin/pkg/pulsar/utils/topic_type.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 utils + +import "github.com/pkg/errors" + +type TopicType string + +const ( + Partitioned TopicType = "partitioned" + NonPartitioned TopicType = "non-partitioned" +) + +func ParseTopicType(topicType string) (TopicType, error) { + switch topicType { + case "partitioned": + return Partitioned, nil + case "non-partitioned": + return NonPartitioned, nil + default: + return "", errors.Errorf("The topic type can only be specified as 'partitioned' or "+ + "'non-partitioned'. Input topic type is '%s'.", topicType) + } +} + +func (t TopicType) String() string { + return string(t) +} From c2488beb587ba9b3845f5ad27527f7123de5c7cf Mon Sep 17 00:00:00 2001 From: Peter Tinti Date: Mon, 17 May 2021 03:04:11 -0400 Subject: [PATCH 181/348] Clone default transport instead of modifying directly (streamnative/pulsarctl#299) * clone default transport instead of modifying directly * bumps golang to v1.13 * updates ci to use go v1.13 --- pulsaradmin/go.mod | 2 +- pulsaradmin/pkg/auth/auth_provider.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index ca119dd8bf..f24447ad2b 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -1,6 +1,6 @@ module github.com/streamnative/pulsar-admin-go -go 1.12 +go 1.13 require ( github.com/99designs/keyring v1.1.6 diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index edac0edddc..18e7f1c639 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -67,7 +67,7 @@ func GetAuthProvider(config *common.Config) (*Provider, error) { } func GetDefaultTransport(config *common.Config) http.RoundTripper { - transport := http.DefaultTransport.(*http.Transport) + transport := http.DefaultTransport.(*http.Transport).Clone() tlsConfig := &tls.Config{ InsecureSkipVerify: config.TLSAllowInsecureConnection, } From 7cf3aff3876c1e41dd5191b2ec0666f9aa475164 Mon Sep 17 00:00:00 2001 From: Nathan Mills Date: Thu, 27 May 2021 19:39:47 -0600 Subject: [PATCH 182/348] Adds the `isOffloaded` field (streamnative/pulsarctl#288) * Adds the `isOffloaded` field * add tests for offloaded field --- pulsaradmin/pkg/pulsar/utils/data.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index f8d6f755b5..fd9589d931 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -261,7 +261,8 @@ type LedgerInfo struct { LedgerID int64 `json:"ledgerId"` Entries int64 `json:"entries"` Size int64 `json:"size"` - Offloaded bool `json:"offloaded"` + Timestamp int64 `json:"timestamp"` + Offloaded bool `json:"isOffloaded"` } type CursorInfo struct { From 60bafa0206a1bf60351b8fc14c800aac6533aa26 Mon Sep 17 00:00:00 2001 From: limingnihao Date: Wed, 2 Jun 2021 09:35:46 +0800 Subject: [PATCH 183/348] Add command topic message ttl. (streamnative/pulsarctl#246) (#348) Add command topic message ttl: * pulsarctl topics get-message-ttl [topic] * pulsarctl topics set-message-ttl [topic] -t [seconds] * pulsarctl topics remove-message-ttl [topic] --- pulsaradmin/pkg/cli/client.go | 22 ++++++++++++++++++++++ pulsaradmin/pkg/pulsar/topic.go | 32 ++++++++++++++++++++++++++++++++ 2 files changed, 54 insertions(+) diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index 465027b24e..2593e3c2a7 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -277,6 +277,28 @@ func (c *Client) PostWithMultiPart(endpoint string, in interface{}, body io.Read return nil } +func (c *Client) PostWithQueryParams(endpoint string, params map[string]string) error { + req, err := c.newRequest(http.MethodPost, endpoint) + if err != nil { + return err + } + if params != nil { + query := req.url.Query() + for k, v := range params { + query.Add(k, v) + } + req.params = query + } + // nolint + resp, err := checkSuccessful(c.doRequest(req)) + if err != nil { + return err + } + defer safeRespClose(resp) + + return nil +} + type request struct { method string contentType string diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index f5865a3431..761a93e045 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -98,6 +98,15 @@ type Topics interface { // CompactStatus checks the status of an ongoing compaction for a topic CompactStatus(utils.TopicName) (utils.LongRunningProcessStatus, error) + + // GetMessageTTL Get the message TTL for a topic + GetMessageTTL(utils.TopicName) (int, error) + + // SetMessageTTL Set the message TTL for a topic + SetMessageTTL(utils.TopicName, int) error + + // RemoveMessageTTL Remove the message TTL for a topic + RemoveMessageTTL(utils.TopicName) error } type topics struct { @@ -301,3 +310,26 @@ func (t *topics) CompactStatus(topic utils.TopicName) (utils.LongRunningProcessS err := t.pulsar.Client.Get(endpoint, &status) return status, err } + +func (t *topics) GetMessageTTL(topic utils.TopicName) (int, error) { + var ttl int + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "messageTTL") + err := t.pulsar.Client.Get(endpoint, &ttl) + return ttl, err +} + +func (t *topics) SetMessageTTL(topic utils.TopicName, messageTTL int) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "messageTTL") + var params = make(map[string]string) + params["messageTTL"] = strconv.Itoa(messageTTL) + err := t.pulsar.Client.PostWithQueryParams(endpoint, params) + return err +} + +func (t *topics) RemoveMessageTTL(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "messageTTL") + var params = make(map[string]string) + params["messageTTL"] = strconv.Itoa(0) + err := t.pulsar.Client.DeleteWithQueryParams(endpoint, params) + return err +} From 9daf4ab5bf36e4e7afe4102a84949b0e2b04d562 Mon Sep 17 00:00:00 2001 From: limingnihao Date: Wed, 2 Jun 2021 13:57:33 +0800 Subject: [PATCH 184/348] Add command topic max producers. (streamnative/pulsarctl#246) (#353) Add command topic max number of producers: * pulsarctl topics get-max-producers [topic] * pulsarctl topics set-max-producers [topic] -p [max] * pulsarctl topics remove-max-producers [topic] --- pulsaradmin/pkg/pulsar/topic.go | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 761a93e045..152821cbf4 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -107,6 +107,15 @@ type Topics interface { // RemoveMessageTTL Remove the message TTL for a topic RemoveMessageTTL(utils.TopicName) error + + // GetMaxProducers Get max number of producers for a topic + GetMaxProducers(utils.TopicName) (int, error) + + // SetMaxProducers Set max number of producers for a topic + SetMaxProducers(utils.TopicName, int) error + + // RemoveMaxProducers Remove max number of producers for a topic + RemoveMaxProducers(utils.TopicName) error } type topics struct { @@ -333,3 +342,22 @@ func (t *topics) RemoveMessageTTL(topic utils.TopicName) error { err := t.pulsar.Client.DeleteWithQueryParams(endpoint, params) return err } + +func (t *topics) GetMaxProducers(topic utils.TopicName) (int, error) { + var maxProducers int + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxProducers") + err := t.pulsar.Client.Get(endpoint, &maxProducers) + return maxProducers, err +} + +func (t *topics) SetMaxProducers(topic utils.TopicName, maxProducers int) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxProducers") + err := t.pulsar.Client.Post(endpoint, &maxProducers) + return err +} + +func (t *topics) RemoveMaxProducers(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxProducers") + err := t.pulsar.Client.Delete(endpoint) + return err +} From 8690a147727925f35564969b013f44c7c481d15c Mon Sep 17 00:00:00 2001 From: limingnihao Date: Wed, 2 Jun 2021 19:03:44 +0800 Subject: [PATCH 185/348] Add command topic max consumers. (streamnative/pulsarctl#246) (#355) Add command topic max number of consumers: * pulsarctl topics get-max-consumers [topic] * pulsarctl topics set-max-consumers [topic] -c [max] * pulsarctl topics remove-max-consumers [topic] --- pulsaradmin/pkg/pulsar/topic.go | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 152821cbf4..ac5ca3dabc 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -116,6 +116,15 @@ type Topics interface { // RemoveMaxProducers Remove max number of producers for a topic RemoveMaxProducers(utils.TopicName) error + + // GetMaxConsumers Get max number of consumers for a topic + GetMaxConsumers(utils.TopicName) (int, error) + + // SetMaxConsumers Set max number of consumers for a topic + SetMaxConsumers(utils.TopicName, int) error + + // RemoveMaxConsumers Remove max number of consumers for a topic + RemoveMaxConsumers(utils.TopicName) error } type topics struct { @@ -361,3 +370,22 @@ func (t *topics) RemoveMaxProducers(topic utils.TopicName) error { err := t.pulsar.Client.Delete(endpoint) return err } + +func (t *topics) GetMaxConsumers(topic utils.TopicName) (int, error) { + var maxConsumers int + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxConsumers") + err := t.pulsar.Client.Get(endpoint, &maxConsumers) + return maxConsumers, err +} + +func (t *topics) SetMaxConsumers(topic utils.TopicName, maxConsumers int) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxConsumers") + err := t.pulsar.Client.Post(endpoint, &maxConsumers) + return err +} + +func (t *topics) RemoveMaxConsumers(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxConsumers") + err := t.pulsar.Client.Delete(endpoint) + return err +} From 1f31f3a88b418a67262b777f7f20bc3fb4cfb7f5 Mon Sep 17 00:00:00 2001 From: limingnihao Date: Thu, 3 Jun 2021 20:58:11 +0800 Subject: [PATCH 186/348] Add command topic max unacked messages per consumer. (streamnative/pulsarctl#246) (#358) Add command topic max unacked messages per consumer: * pulsarctl topics get-max-unacked-messages-per-consumer [topic] * pulsarctl topics set-max-unacked-messages-per-consumer [topic] -m [max] * pulsarctl topics remove-max-unacked-messages-per-consumer [topic] --- pulsaradmin/pkg/pulsar/topic.go | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index ac5ca3dabc..3b534496ac 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -125,6 +125,15 @@ type Topics interface { // RemoveMaxConsumers Remove max number of consumers for a topic RemoveMaxConsumers(utils.TopicName) error + + // GetMaxUnackMessagesPerConsumer Get max unacked messages policy on consumer for a topic + GetMaxUnackMessagesPerConsumer(utils.TopicName) (int, error) + + // SetMaxUnackMessagesPerConsumer Set max unacked messages policy on consumer for a topic + SetMaxUnackMessagesPerConsumer(utils.TopicName, int) error + + // RemoveMaxUnackMessagesPerConsumer Remove max unacked messages policy on consumer for a topic + RemoveMaxUnackMessagesPerConsumer(utils.TopicName) error } type topics struct { @@ -389,3 +398,20 @@ func (t *topics) RemoveMaxConsumers(topic utils.TopicName) error { err := t.pulsar.Client.Delete(endpoint) return err } + +func (t *topics) GetMaxUnackMessagesPerConsumer(topic utils.TopicName) (int, error) { + var maxNum int + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxUnackedMessagesOnConsumer") + err := t.pulsar.Client.Get(endpoint, &maxNum) + return maxNum, err +} + +func (t *topics) SetMaxUnackMessagesPerConsumer(topic utils.TopicName, maxUnackedNum int) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxUnackedMessagesOnConsumer") + return t.pulsar.Client.Post(endpoint, &maxUnackedNum) +} + +func (t *topics) RemoveMaxUnackMessagesPerConsumer(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxUnackedMessagesOnConsumer") + return t.pulsar.Client.Delete(endpoint) +} From 6c6c5013d22c54ac46ec39e554bf08a949117652 Mon Sep 17 00:00:00 2001 From: limingnihao Date: Mon, 7 Jun 2021 11:17:03 +0800 Subject: [PATCH 187/348] Add command topic max unacked messages per subscription. (streamnative/pulsarctl#246) (#361) Add command topic max unacked messages per subscription: * pulsarctl topics get-max-unacked-messages-per-subscription [topic] * pulsarctl topics set-max-unacked-messages-per-subscription [topic] -m [max] * pulsarctl topics remove-max-unacked-messages-per-subscription [topic] --- pulsaradmin/pkg/pulsar/topic.go | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 3b534496ac..aee96f655a 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -134,6 +134,15 @@ type Topics interface { // RemoveMaxUnackMessagesPerConsumer Remove max unacked messages policy on consumer for a topic RemoveMaxUnackMessagesPerConsumer(utils.TopicName) error + + // GetMaxUnackMessagesPerSubscription Get max unacked messages policy on subscription for a topic + GetMaxUnackMessagesPerSubscription(utils.TopicName) (int, error) + + // SetMaxUnackMessagesPerSubscription Set max unacked messages policy on subscription for a topic + SetMaxUnackMessagesPerSubscription(utils.TopicName, int) error + + // RemoveMaxUnackMessagesPerSubscription Remove max unacked messages policy on subscription for a topic + RemoveMaxUnackMessagesPerSubscription(utils.TopicName) error } type topics struct { @@ -415,3 +424,20 @@ func (t *topics) RemoveMaxUnackMessagesPerConsumer(topic utils.TopicName) error endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxUnackedMessagesOnConsumer") return t.pulsar.Client.Delete(endpoint) } + +func (t *topics) GetMaxUnackMessagesPerSubscription(topic utils.TopicName) (int, error) { + var maxNum int + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxUnackedMessagesOnSubscription") + err := t.pulsar.Client.Get(endpoint, &maxNum) + return maxNum, err +} + +func (t *topics) SetMaxUnackMessagesPerSubscription(topic utils.TopicName, maxUnackedNum int) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxUnackedMessagesOnSubscription") + return t.pulsar.Client.Post(endpoint, &maxUnackedNum) +} + +func (t *topics) RemoveMaxUnackMessagesPerSubscription(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxUnackedMessagesOnSubscription") + return t.pulsar.Client.Delete(endpoint) +} From 0aeb9f7a61e7dbc048c3314ac35b95a92f7343a6 Mon Sep 17 00:00:00 2001 From: limingnihao Date: Wed, 9 Jun 2021 17:11:36 +0800 Subject: [PATCH 188/348] Add command topic Persistence Policy. (streamnative/pulsarctl#246) (#363) --- pulsaradmin/pkg/pulsar/topic.go | 26 ++++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/utils/data.go | 7 +++++++ 2 files changed, 33 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index aee96f655a..54b0e8cb3d 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -143,6 +143,15 @@ type Topics interface { // RemoveMaxUnackMessagesPerSubscription Remove max unacked messages policy on subscription for a topic RemoveMaxUnackMessagesPerSubscription(utils.TopicName) error + + // GetPersistence Get the persistence policies for a topic + GetPersistence(utils.TopicName) (*utils.PersistenceData, error) + + // SetPersistence Set the persistence policies for a topic + SetPersistence(utils.TopicName, utils.PersistenceData) error + + // RemovePersistence Remove the persistence policies for a topic + RemovePersistence(utils.TopicName) error } type topics struct { @@ -441,3 +450,20 @@ func (t *topics) RemoveMaxUnackMessagesPerSubscription(topic utils.TopicName) er endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "maxUnackedMessagesOnSubscription") return t.pulsar.Client.Delete(endpoint) } + +func (t *topics) GetPersistence(topic utils.TopicName) (*utils.PersistenceData, error) { + var persistenceData utils.PersistenceData + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "persistence") + err := t.pulsar.Client.Get(endpoint, &persistenceData) + return &persistenceData, err +} + +func (t *topics) SetPersistence(topic utils.TopicName, persistenceData utils.PersistenceData) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "persistence") + return t.pulsar.Client.Post(endpoint, &persistenceData) +} + +func (t *topics) RemovePersistence(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "persistence") + return t.pulsar.Client.Delete(endpoint) +} diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index fd9589d931..d171178d09 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -362,3 +362,10 @@ type ResourceQuotaData struct { Memory int64 `json:"memory"` Dynamic bool `json:"dynamic"` } + +type PersistenceData struct { + BookkeeperEnsemble int64 `json:"bookkeeperEnsemble"` + BookkeeperWriteQuorum int64 `json:"bookkeeperWriteQuorum"` + BookkeeperAckQuorum int64 `json:"bookkeeperAckQuorum"` + ManagedLedgerMaxMarkDeleteRate float64 `json:"managedLedgerMaxMarkDeleteRate"` +} From d644161697acd7c3b4b3913f4564f36f69a0cc44 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 14 Jun 2021 22:59:54 +0800 Subject: [PATCH 189/348] Support using Json to configure the token auth provider (streamnative/pulsarctl#368) * Support using Json to configure the token auth provider --- *Motivation* In the pulsar-cient-go, we support using the json format to parse the auth-params. * Fix the project style check --- pulsaradmin/pkg/auth/auth_provider.go | 2 ++ pulsaradmin/pkg/auth/token.go | 33 +++++++++++++++++++-------- 2 files changed, 25 insertions(+), 10 deletions(-) diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index 18e7f1c639..4136416644 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -47,6 +47,8 @@ func GetAuthProvider(config *common.Config) (*Provider, error) { case TLSPluginName: provider, err = NewAuthenticationTLSFromAuthParams(config.AuthParams, defaultTransport) case TokenPluginName: + fallthrough + case TokePluginShortName: provider, err = NewAuthenticationTokenFromAuthParams(config.AuthParams, defaultTransport) default: switch { diff --git a/pulsaradmin/pkg/auth/token.go b/pulsaradmin/pkg/auth/token.go index 18088a5427..5b6eed6e8e 100644 --- a/pulsaradmin/pkg/auth/token.go +++ b/pulsaradmin/pkg/auth/token.go @@ -18,6 +18,7 @@ package auth import ( + "encoding/json" "fmt" "io/ioutil" "net/http" @@ -27,11 +28,16 @@ import ( ) const ( - tokenPrefix = "token:" - filePrefix = "file:" - TokenPluginName = "org.apache.pulsar.client.impl.auth.AuthenticationToken" + tokenPrefix = "token:" + filePrefix = "file:" + TokenPluginName = "org.apache.pulsar.client.impl.auth.AuthenticationToken" + TokePluginShortName = "token" ) +type Token struct { + Token string `json:"token"` +} + type TokenAuthProvider struct { T http.RoundTripper token string @@ -59,13 +65,20 @@ func NewAuthenticationTokenFromAuthParams(encodedAuthParam string, transport http.RoundTripper) (*TokenAuthProvider, error) { var tokenAuthProvider *TokenAuthProvider var err error - switch { - case strings.HasPrefix(encodedAuthParam, tokenPrefix): - tokenAuthProvider, err = NewAuthenticationToken(strings.TrimPrefix(encodedAuthParam, tokenPrefix), transport) - case strings.HasPrefix(encodedAuthParam, filePrefix): - tokenAuthProvider, err = NewAuthenticationTokenFromFile(strings.TrimPrefix(encodedAuthParam, filePrefix), transport) - default: - tokenAuthProvider, err = NewAuthenticationToken(encodedAuthParam, transport) + + var tokenJSON Token + err = json.Unmarshal([]byte(encodedAuthParam), &tokenJSON) + if err != nil { + switch { + case strings.HasPrefix(encodedAuthParam, tokenPrefix): + tokenAuthProvider, err = NewAuthenticationToken(strings.TrimPrefix(encodedAuthParam, tokenPrefix), transport) + case strings.HasPrefix(encodedAuthParam, filePrefix): + tokenAuthProvider, err = NewAuthenticationTokenFromFile(strings.TrimPrefix(encodedAuthParam, filePrefix), transport) + default: + tokenAuthProvider, err = NewAuthenticationToken(encodedAuthParam, transport) + } + } else { + tokenAuthProvider, err = NewAuthenticationToken(tokenJSON.Token, transport) } return tokenAuthProvider, err } From d5b6ddfb627d6fdc8ae5147e5b9662ca9c77f694 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Tue, 15 Jun 2021 15:03:00 +0800 Subject: [PATCH 190/348] Support use short auth name to configure auth (streamnative/pulsarctl#371) * Support use short auth name to configure auth --- *Motivation* Support use `tls` to configure the TLS provider * Fix the check style issue --- pulsaradmin/pkg/auth/auth_provider.go | 2 ++ pulsaradmin/pkg/auth/tls.go | 34 ++++++++++++++++++++------- 2 files changed, 27 insertions(+), 9 deletions(-) diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index 4136416644..7c2e358583 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -44,6 +44,8 @@ func GetAuthProvider(config *common.Config) (*Provider, error) { defaultTransport := GetDefaultTransport(config) var err error switch config.AuthPlugin { + case TLSPluginShortName: + fallthrough case TLSPluginName: provider, err = NewAuthenticationTLSFromAuthParams(config.AuthParams, defaultTransport) case TokenPluginName: diff --git a/pulsaradmin/pkg/auth/tls.go b/pulsaradmin/pkg/auth/tls.go index 3ac2e303f0..7c11fe27e7 100644 --- a/pulsaradmin/pkg/auth/tls.go +++ b/pulsaradmin/pkg/auth/tls.go @@ -19,14 +19,21 @@ package auth import ( "crypto/tls" + "encoding/json" "net/http" "strings" ) const ( - TLSPluginName = "org.apache.pulsar.client.impl.auth.AuthenticationTls" + TLSPluginName = "org.apache.pulsar.client.impl.auth.AuthenticationTls" + TLSPluginShortName = "tls" ) +type TLS struct { + TLSCertFile string `json:"tlsCertFile"` + TLSKeyFile string `json:"tlsKeyFile"` +} + type TLSAuthProvider struct { certificatePath string privateKeyPath string @@ -51,16 +58,25 @@ func NewAuthenticationTLSFromAuthParams(encodedAuthParams string, transport http.RoundTripper) (*TLSAuthProvider, error) { var certificatePath string var privateKeyPath string - parts := strings.Split(encodedAuthParams, ",") - for _, part := range parts { - kv := strings.Split(part, ":") - switch kv[0] { - case "tlsCertFile": - certificatePath = kv[1] - case "tlsKeyFile": - privateKeyPath = kv[1] + + var tlsJSON TLS + err := json.Unmarshal([]byte(encodedAuthParams), &tlsJSON) + if err != nil { + parts := strings.Split(encodedAuthParams, ",") + for _, part := range parts { + kv := strings.Split(part, ":") + switch kv[0] { + case "tlsCertFile": + certificatePath = kv[1] + case "tlsKeyFile": + privateKeyPath = kv[1] + } } + } else { + certificatePath = tlsJSON.TLSCertFile + privateKeyPath = tlsJSON.TLSKeyFile } + return NewAuthenticationTLS(certificatePath, privateKeyPath, transport) } From 93c0e4523a093ee3011e7cacf9664007da37dad2 Mon Sep 17 00:00:00 2001 From: StreamNative Bot <44651360+streamnativebot@users.noreply.github.com> Date: Thu, 17 Jun 2021 04:18:53 -0700 Subject: [PATCH 191/348] Bump version to 2.8.0-rc-202106151929 (streamnative/pulsarctl#372) * Created by streamnative-ci * Fix the failure tests * Fix the tests * Fix the failure tests * Fix the failure tests * Fix the failure tests Co-authored-by: streamnativebot Co-authored-by: Yong Zhang --- pulsaradmin/pkg/pulsar/utils/backlog_quota.go | 12 +++++++----- pulsaradmin/pkg/pulsar/utils/data.go | 1 + 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/utils/backlog_quota.go b/pulsaradmin/pkg/pulsar/utils/backlog_quota.go index ccfce493c4..2d14f809b6 100644 --- a/pulsaradmin/pkg/pulsar/utils/backlog_quota.go +++ b/pulsaradmin/pkg/pulsar/utils/backlog_quota.go @@ -20,14 +20,16 @@ package utils import "github.com/pkg/errors" type BacklogQuota struct { - Limit int64 `json:"limit"` - Policy RetentionPolicy `json:"policy"` + LimitTime int64 `json:"limitTime"` + LimitSize int64 `json:"limitSize"` + Policy RetentionPolicy `json:"policy"` } -func NewBacklogQuota(limit int64, policy RetentionPolicy) BacklogQuota { +func NewBacklogQuota(limitSize int64, limitTime int64, policy RetentionPolicy) BacklogQuota { return BacklogQuota{ - Limit: limit, - Policy: policy, + LimitSize: limitSize, + LimitTime: limitTime, + Policy: policy, } } diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index d171178d09..2aa8e8d2ae 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -171,6 +171,7 @@ type NamespacesData struct { ClusterIds string `json:"clusterIds"` RetentionTimeStr string `json:"retentionTimeStr"` LimitStr string `json:"limitStr"` + LimitTime int64 `json:"limitTime"` PolicyStr string `json:"policyStr"` AntiAffinityGroup string `json:"antiAffinityGroup"` Tenant string `json:"tenant"` From 6f562454e27fd6a387c3718fce64a9b212b84c83 Mon Sep 17 00:00:00 2001 From: limingnihao Date: Fri, 18 Jun 2021 09:15:14 +0800 Subject: [PATCH 192/348] Add command topic Delayed Delivery Policies. (streamnative/pulsarctl#246) (#374) Add command topic Delayed Delivery Policies: - pulsarctl topics get-delayed-delivery [topic] - pulsarctl topics set-delayed-delivery [topic] -t 22s -e - pulsarctl topics remove-delayed-delivery [topic] --- pulsaradmin/pkg/pulsar/topic.go | 26 ++++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/utils/data.go | 11 +++++++++++ 2 files changed, 37 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 54b0e8cb3d..a92e8a8d01 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -152,6 +152,15 @@ type Topics interface { // RemovePersistence Remove the persistence policies for a topic RemovePersistence(utils.TopicName) error + + // GetDelayedDelivery Get the delayed delivery policy for a topic + GetDelayedDelivery(utils.TopicName) (*utils.DelayedDeliveryData, error) + + // SetDelayedDelivery Set the delayed delivery policy on a topic + SetDelayedDelivery(utils.TopicName, utils.DelayedDeliveryData) error + + // RemoveDelayedDelivery Remove the delayed delivery policy on a topic + RemoveDelayedDelivery(utils.TopicName) error } type topics struct { @@ -467,3 +476,20 @@ func (t *topics) RemovePersistence(topic utils.TopicName) error { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "persistence") return t.pulsar.Client.Delete(endpoint) } + +func (t *topics) GetDelayedDelivery(topic utils.TopicName) (*utils.DelayedDeliveryData, error) { + var delayedDeliveryData utils.DelayedDeliveryData + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "delayedDelivery") + err := t.pulsar.Client.Get(endpoint, &delayedDeliveryData) + return &delayedDeliveryData, err +} + +func (t *topics) SetDelayedDelivery(topic utils.TopicName, delayedDeliveryData utils.DelayedDeliveryData) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "delayedDelivery") + return t.pulsar.Client.Post(endpoint, &delayedDeliveryData) +} + +func (t *topics) RemoveDelayedDelivery(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "delayedDelivery") + return t.pulsar.Client.Delete(endpoint) +} diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index 2aa8e8d2ae..95eb733072 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -370,3 +370,14 @@ type PersistenceData struct { BookkeeperAckQuorum int64 `json:"bookkeeperAckQuorum"` ManagedLedgerMaxMarkDeleteRate float64 `json:"managedLedgerMaxMarkDeleteRate"` } + +type DelayedDeliveryCmdData struct { + Enable bool `json:"enable"` + Disable bool `json:"disable"` + DelayedDeliveryTimeStr string `json:"delayedDeliveryTimeStr"` +} + +type DelayedDeliveryData struct { + TickTime float64 `json:"tickTime"` + Active bool `json:"active"` +} From 1d7be34c9eabfcf489e0192c9ac643f5a98b5ff5 Mon Sep 17 00:00:00 2001 From: limingnihao Date: Tue, 20 Jul 2021 14:44:12 +0800 Subject: [PATCH 193/348] Add command topic message dispatch rate. (streamnative/pulsarctl#246) (#397) Add command topic Message Dispatch Rate: > Pulsarctl does not support 2-letter shorthand, so use the full length. - pulsarctl topics get-dispatch-rate [topic] - pulsarctl topics set-dispatch-rate [topic] --msg-dispatch-rate 1 --byte-dispatch-rate 2 --dispatch-rate-period 3 --relative-to-publish-rate - pulsarctl topics remove-dispatch-rate [topic] --- pulsaradmin/pkg/pulsar/topic.go | 26 ++++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/utils/data.go | 7 +++++++ 2 files changed, 33 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index a92e8a8d01..b59a034050 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -161,6 +161,15 @@ type Topics interface { // RemoveDelayedDelivery Remove the delayed delivery policy on a topic RemoveDelayedDelivery(utils.TopicName) error + + // GetDispatchRate Get message dispatch rate for a topic + GetDispatchRate(utils.TopicName) (*utils.DispatchRateData, error) + + // SetDispatchRate Set message dispatch rate for a topic + SetDispatchRate(utils.TopicName, utils.DispatchRateData) error + + // RemoveDispatchRate Remove message dispatch rate for a topic + RemoveDispatchRate(utils.TopicName) error } type topics struct { @@ -493,3 +502,20 @@ func (t *topics) RemoveDelayedDelivery(topic utils.TopicName) error { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "delayedDelivery") return t.pulsar.Client.Delete(endpoint) } + +func (t *topics) GetDispatchRate(topic utils.TopicName) (*utils.DispatchRateData, error) { + var dispatchRateData utils.DispatchRateData + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "dispatchRate") + err := t.pulsar.Client.Get(endpoint, &dispatchRateData) + return &dispatchRateData, err +} + +func (t *topics) SetDispatchRate(topic utils.TopicName, dispatchRateData utils.DispatchRateData) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "dispatchRate") + return t.pulsar.Client.Post(endpoint, &dispatchRateData) +} + +func (t *topics) RemoveDispatchRate(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "dispatchRate") + return t.pulsar.Client.Delete(endpoint) +} diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index 95eb733072..e42c6fef9f 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -381,3 +381,10 @@ type DelayedDeliveryData struct { TickTime float64 `json:"tickTime"` Active bool `json:"active"` } + +type DispatchRateData struct { + DispatchThrottlingRateInMsg int64 `json:"dispatchThrottlingRateInMsg"` + DispatchThrottlingRateInByte int64 `json:"dispatchThrottlingRateInByte"` + RatePeriodInSecond int64 `json:"ratePeriodInSecond"` + RelativeToPublishRate bool `json:"relativeToPublishRate"` +} From bbb7854ced74725bef0137f5dfa118b8c7ce953f Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 29 Jul 2021 20:44:00 +0800 Subject: [PATCH 194/348] Fix the 400 error when create function (streamnative/pulsarctl#405) * Fix the 400 error when create function --- *Motivation* When using pulsarctl create functions, pulsarctl always get 400 bac request error. That cause by we pass the unneeded field in the request. This PR ignore the function configurations in the request if the field is empty. *Modifications* - Ignore the empty field in the function configuration data when transform to json *Verify this change* Update the function test image to running the created test --- pulsaradmin/pkg/cli/client.go | 1 + .../pkg/pulsar/utils/function_confg.go | 64 +++++++++---------- 2 files changed, 33 insertions(+), 32 deletions(-) diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index 2593e3c2a7..29629ee89d 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -393,6 +393,7 @@ func responseError(resp *http.Response) error { return e } + e.Reason = string(body) err = json.Unmarshal(body, &e) if err != nil { e.Reason = string(body) diff --git a/pulsaradmin/pkg/pulsar/utils/function_confg.go b/pulsaradmin/pkg/pulsar/utils/function_confg.go index fc663aadc1..d3c873bf19 100644 --- a/pulsaradmin/pkg/pulsar/utils/function_confg.go +++ b/pulsaradmin/pkg/pulsar/utils/function_confg.go @@ -24,54 +24,54 @@ const ( ) type FunctionConfig struct { - TimeoutMs *int64 `json:"timeoutMs" yaml:"timeoutMs"` - TopicsPattern *string `json:"topicsPattern" yaml:"topicsPattern"` + TimeoutMs *int64 `json:"timeoutMs,omitempty" yaml:"timeoutMs"` + TopicsPattern *string `json:"topicsPattern,omitempty" yaml:"topicsPattern"` // Whether the subscriptions the functions created/used should be deleted when the functions is deleted - CleanupSubscription bool `json:"cleanupSubscription" yaml:"cleanupSubscription"` - RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` - AutoAck bool `json:"autoAck" yaml:"autoAck"` - Parallelism int `json:"parallelism" yaml:"parallelism"` - MaxMessageRetries *int `json:"maxMessageRetries" yaml:"maxMessageRetries"` + CleanupSubscription bool `json:"cleanupSubscription,omitempty" yaml:"cleanupSubscription"` + RetainOrdering bool `json:"retainOrdering,omitempty" yaml:"retainOrdering"` + AutoAck bool `json:"autoAck,omitempty" yaml:"autoAck"` + Parallelism int `json:"parallelism,omitempty" yaml:"parallelism"` + MaxMessageRetries *int `json:"maxMessageRetries,omitempty" yaml:"maxMessageRetries"` - Output string `json:"output" yaml:"output"` + Output string `json:"output,omitempty" yaml:"output"` - OutputSerdeClassName string `json:"outputSerdeClassName" yaml:"outputSerdeClassName"` - LogTopic string `json:"logTopic" yaml:"logTopic"` - ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` + OutputSerdeClassName string `json:"outputSerdeClassName,omitempty" yaml:"outputSerdeClassName"` + LogTopic string `json:"logTopic,omitempty" yaml:"logTopic"` + ProcessingGuarantees string `json:"processingGuarantees,omitempty" yaml:"processingGuarantees"` // Represents either a builtin schema type (eg: 'avro', 'json', etc) or the class name for a Schema implementation - OutputSchemaType string `json:"outputSchemaType" yaml:"outputSchemaType"` + OutputSchemaType string `json:"outputSchemaType,omitempty" yaml:"outputSchemaType"` - Runtime string `json:"runtime" yaml:"runtime"` - DeadLetterTopic string `json:"deadLetterTopic" yaml:"deadLetterTopic"` - SubName string `json:"subName" yaml:"subName"` - FQFN string `json:"fqfn" yaml:"fqfn"` - Jar *string `json:"jar" yaml:"jar"` - Py *string `json:"py" yaml:"py"` - Go *string `json:"go" yaml:"go"` + Runtime string `json:"runtime,omitempty" yaml:"runtime"` + DeadLetterTopic string `json:"deadLetterTopic,omitempty" yaml:"deadLetterTopic"` + SubName string `json:"subName,omitempty" yaml:"subName"` + FQFN string `json:"fqfn,omitempty" yaml:"fqfn"` + Jar *string `json:"jar,omitempty" yaml:"jar"` + Py *string `json:"py,omitempty" yaml:"py"` + Go *string `json:"go,omitempty" yaml:"go"` // Any flags that you want to pass to the runtime. // note that in thread mode, these flags will have no impact - RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` + RuntimeFlags string `json:"runtimeFlags,omitempty" yaml:"runtimeFlags"` - Tenant string `json:"tenant" yaml:"tenant"` - Namespace string `json:"namespace" yaml:"namespace"` - Name string `json:"name" yaml:"name"` - ClassName string `json:"className" yaml:"className"` + Tenant string `json:"tenant,omitempty" yaml:"tenant"` + Namespace string `json:"namespace,omitempty" yaml:"namespace"` + Name string `json:"name,omitempty" yaml:"name"` + ClassName string `json:"className,omitempty" yaml:"className"` - Resources *Resources `json:"resources" yaml:"resources"` - WindowConfig *WindowConfig `json:"windowConfig" yaml:"windowConfig"` - Inputs []string `json:"inputs" yaml:"inputs"` - UserConfig map[string]interface{} `json:"userConfig" yaml:"userConfig"` - CustomSerdeInputs map[string]string `json:"customSerdeInputs" yaml:"customSerdeInputs"` - CustomSchemaInputs map[string]string `json:"customSchemaInputs" yaml:"customSchemaInputs"` + Resources *Resources `json:"resources,omitempty" yaml:"resources"` + WindowConfig *WindowConfig `json:"windowConfig,omitempty" yaml:"windowConfig"` + Inputs []string `json:"inputs,omitempty" yaml:"inputs"` + UserConfig map[string]interface{} `json:"userConfig,omitempty" yaml:"userConfig"` + CustomSerdeInputs map[string]string `json:"customSerdeInputs,omitempty" yaml:"customSerdeInputs"` + CustomSchemaInputs map[string]string `json:"customSchemaInputs,omitempty" yaml:"customSchemaInputs"` // A generalized way of specifying inputs - InputSpecs map[string]ConsumerConfig `json:"inputSpecs" yaml:"inputSpecs"` + InputSpecs map[string]ConsumerConfig `json:"inputSpecs,omitempty" yaml:"inputSpecs"` // This is a map of secretName(aka how the secret is going to be // accessed in the function via context) to an object that // encapsulates how the secret is fetched by the underlying // secrets provider. The type of an value here can be found by the // SecretProviderConfigurator.getSecretObjectType() method. - Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` + Secrets map[string]interface{} `json:"secrets,omitempty" yaml:"secrets"` } From 9e9f38271f98225ab6e003dc7b63bd407da676d9 Mon Sep 17 00:00:00 2001 From: limingnihao Date: Wed, 4 Aug 2021 11:12:10 +0800 Subject: [PATCH 195/348] Add command topic Deduplication(streamnative/pulsarctl#246) (#408) * Add command topic Deduplication(streamnative/pulsarctl#246) - pulsarctl topics get-deduplication [topic] - pulsarctl topics set-deduplication [topic] -e - pulsarctl topics remove-deduplication [topic] * Modify prompt Signed-off-by: limingnihao --- pulsaradmin/pkg/pulsar/topic.go | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index b59a034050..69b718c1fe 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -170,6 +170,15 @@ type Topics interface { // RemoveDispatchRate Remove message dispatch rate for a topic RemoveDispatchRate(utils.TopicName) error + + // GetDeduplicationStatus Get the deduplication policy for a topic + GetDeduplicationStatus(utils.TopicName) (bool, error) + + // SetDeduplicationStatus Set the deduplication policy for a topic + SetDeduplicationStatus(utils.TopicName, bool) error + + // RemoveDeduplicationStatus Remove the deduplication policy for a topic + RemoveDeduplicationStatus(utils.TopicName) error } type topics struct { @@ -519,3 +528,19 @@ func (t *topics) RemoveDispatchRate(topic utils.TopicName) error { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "dispatchRate") return t.pulsar.Client.Delete(endpoint) } + +func (t *topics) GetDeduplicationStatus(topic utils.TopicName) (bool, error) { + var enabled bool + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "deduplicationEnabled") + err := t.pulsar.Client.Get(endpoint, &enabled) + return enabled, err +} + +func (t *topics) SetDeduplicationStatus(topic utils.TopicName, enabled bool) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "deduplicationEnabled") + return t.pulsar.Client.Post(endpoint, enabled) +} +func (t *topics) RemoveDeduplicationStatus(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "deduplicationEnabled") + return t.pulsar.Client.Delete(endpoint) +} From a76197d24a6e5387a7f18c8a16b22554e60b8ac8 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 4 Aug 2021 12:09:36 +0800 Subject: [PATCH 196/348] fix: check the parse service url for errors (streamnative/pulsarctl#395) Fix: streamnative/pulsarctl#385 --- pulsaradmin/pkg/cli/client.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index 29629ee89d..3319e9e33d 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -35,7 +35,10 @@ type Client struct { } func (c *Client) newRequest(method, path string) (*request, error) { - base, _ := url.Parse(c.ServiceURL) + base, err := url.Parse(c.ServiceURL) + if err != nil { + return nil, err + } u, err := url.Parse(path) if err != nil { return nil, err From 11358b6a8ead289382d9b226ae51dc822e760276 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 6 Aug 2021 09:18:26 +0800 Subject: [PATCH 197/348] Fix the sink and source tests (streamnative/pulsarctl#411) * Fix the sink and source tests --- Fixes streamnative/pulsarctl#406 *Motivation* Currently, the sink and source tests depend on the kafka and mysql connectors. And those connectors need download from the website. Another thing is we will change the image version when there has a new release. Current testing steps is hard to change to the new version, we need to update all the connectors used in the test, that will introduce unnecessary operations. In the pulsar-all image, we already have the connectors so we can use them directly. So I refactor the sink and source tests to use the connectors in the pulsar image. Using the data-generator connectors for testing the sink and source apis in the pulsarctl. The data-generator connectors doesn't need to depend on other services, it's easier to use in the test. *Modifications* - fix the sink and source tests - update the test scripts to allow test can run by `run-integration-test.sh` - simplify the test steps - update the test workflows *Verify this change* Please pick either of following options. - Pass the tests --- pulsaradmin/pkg/pulsar/sinks.go | 2 +- pulsaradmin/pkg/pulsar/utils/data.go | 90 +++++++++---------- pulsaradmin/pkg/pulsar/utils/sink_config.go | 43 ++++----- pulsaradmin/pkg/pulsar/utils/source_config.go | 28 +++--- 4 files changed, 82 insertions(+), 81 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/sinks.go b/pulsaradmin/pkg/pulsar/sinks.go index 295266b49a..22eb819baf 100644 --- a/pulsaradmin/pkg/pulsar/sinks.go +++ b/pulsaradmin/pkg/pulsar/sinks.go @@ -426,7 +426,7 @@ func (s *sinks) StartSinkWithID(tenant, namespace, sink string, instanceID int) func (s *sinks) GetBuiltInSinks() ([]*utils.ConnectorDefinition, error) { var connectorDefinition []*utils.ConnectorDefinition - endpoint := s.pulsar.endpoint(s.basePath, "builtinSinks") + endpoint := s.pulsar.endpoint(s.basePath, "builtinsinks") err := s.pulsar.Client.Get(endpoint, &connectorDefinition) return connectorDefinition, err } diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index e42c6fef9f..103f097333 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -95,54 +95,54 @@ type TenantData struct { } type SourceData struct { - Tenant string `json:"tenant"` - Namespace string `json:"namespace"` - Name string `json:"name"` - SourceType string `json:"sourceType"` - ProcessingGuarantees string `json:"processingGuarantees"` - DestinationTopicName string `json:"destinationTopicName"` - DeserializationClassName string `json:"deserializationClassName"` - SchemaType string `json:"schemaType"` - Parallelism int `json:"parallelism"` - Archive string `json:"archive"` - ClassName string `json:"className"` - SourceConfigFile string `json:"sourceConfigFile"` - CPU float64 `json:"cpu"` - RAM int64 `json:"ram"` - Disk int64 `json:"disk"` - SourceConfigString string `json:"sourceConfigString"` - - SourceConf *SourceConfig `json:"-"` - InstanceID string `json:"instanceId"` - - UpdateAuthData bool `json:"updateAuthData"` + Tenant string `json:"tenant,omitempty"` + Namespace string `json:"namespace,omitempty"` + Name string `json:"name,omitempty"` + SourceType string `json:"sourceType,omitempty"` + ProcessingGuarantees string `json:"processingGuarantees,omitempty"` + DestinationTopicName string `json:"destinationTopicName,omitempty"` + DeserializationClassName string `json:"deserializationClassName,omitempty"` + SchemaType string `json:"schemaType,omitempty"` + Parallelism int `json:"parallelism,omitempty"` + Archive string `json:"archive,omitempty"` + ClassName string `json:"className,omitempty"` + SourceConfigFile string `json:"sourceConfigFile,omitempty"` + CPU float64 `json:"cpu,omitempty"` + RAM int64 `json:"ram,omitempty"` + Disk int64 `json:"disk,omitempty"` + SourceConfigString string `json:"sourceConfigString,omitempty"` + + SourceConf *SourceConfig `json:"-,omitempty"` + InstanceID string `json:"instanceId,omitempty"` + + UpdateAuthData bool `json:"updateAuthData,omitempty"` } type SinkData struct { - UpdateAuthData bool `json:"updateAuthData"` - RetainOrdering bool `json:"retainOrdering"` - AutoAck bool `json:"autoAck"` - Parallelism int `json:"parallelism"` - RAM int64 `json:"ram"` - Disk int64 `json:"disk"` - TimeoutMs int64 `json:"timeoutMs"` - CPU float64 `json:"cpu"` - Tenant string `json:"tenant"` - Namespace string `json:"namespace"` - Name string `json:"name"` - SinkType string `json:"sinkType"` - Inputs string `json:"inputs"` - TopicsPattern string `json:"topicsPattern"` - SubsName string `json:"subsName"` - CustomSerdeInputString string `json:"customSerdeInputString"` - CustomSchemaInputString string `json:"customSchemaInputString"` - ProcessingGuarantees string `json:"processingGuarantees"` - Archive string `json:"archive"` - ClassName string `json:"className"` - SinkConfigFile string `json:"sinkConfigFile"` - SinkConfigString string `json:"sinkConfigString"` - InstanceID string `json:"instanceId"` - SinkConf *SinkConfig `json:"-"` + UpdateAuthData bool `json:"updateAuthData,omitempty"` + RetainOrdering bool `json:"retainOrdering,omitempty"` + AutoAck bool `json:"autoAck,omitempty"` + Parallelism int `json:"parallelism,omitempty"` + RAM int64 `json:"ram,omitempty"` + Disk int64 `json:"disk,omitempty"` + TimeoutMs int64 `json:"timeoutMs,omitempty"` + CPU float64 `json:"cpu,omitempty"` + Tenant string `json:"tenant,omitempty"` + Namespace string `json:"namespace,omitempty"` + Name string `json:"name,omitempty"` + SinkType string `json:"sinkType,omitempty"` + Inputs string `json:"inputs,omitempty"` + TopicsPattern string `json:"topicsPattern,omitempty"` + SubsName string `json:"subsName,omitempty"` + CustomSerdeInputString string `json:"customSerdeInputString,omitempty"` + CustomSchemaInputString string `json:"customSchemaInputString,omitempty"` + ProcessingGuarantees string `json:"processingGuarantees,omitempty"` + Archive string `json:"archive,omitempty"` + ClassName string `json:"className,omitempty"` + SinkConfigFile string `json:"sinkConfigFile,omitempty"` + SinkConfigString string `json:"sinkConfigString,omitempty"` + InstanceID string `json:"instanceId,omitempty"` + SinkConf *SinkConfig `json:"-,omitempty"` } // Topic data diff --git a/pulsaradmin/pkg/pulsar/utils/sink_config.go b/pulsaradmin/pkg/pulsar/utils/sink_config.go index efa13d31c5..aaaf7dc46f 100644 --- a/pulsaradmin/pkg/pulsar/utils/sink_config.go +++ b/pulsaradmin/pkg/pulsar/utils/sink_config.go @@ -18,35 +18,36 @@ package utils type SinkConfig struct { - TopicsPattern *string `json:"topicsPattern" yaml:"topicsPattern"` - Resources *Resources `json:"resources" yaml:"resources"` - TimeoutMs *int64 `json:"timeoutMs" yaml:"timeoutMs"` + TopicsPattern *string `json:"topicsPattern,omitempty" yaml:"topicsPattern"` + Resources *Resources `json:"resources,omitempty" yaml:"resources"` + TimeoutMs *int64 `json:"timeoutMs,omitempty" yaml:"timeoutMs"` // Whether the subscriptions the functions created/used should be deleted when the functions is deleted - CleanupSubscription bool `json:"cleanupSubscription" yaml:"cleanupSubscription"` + CleanupSubscription bool `json:"cleanupSubscription,omitempty" yaml:"cleanupSubscription"` - RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` - AutoAck bool `json:"autoAck" yaml:"autoAck"` - Parallelism int `json:"parallelism" yaml:"parallelism"` - Tenant string `json:"tenant" yaml:"tenant"` - Namespace string `json:"namespace" yaml:"namespace"` - Name string `json:"name" yaml:"name"` - ClassName string `json:"className" yaml:"className"` + RetainOrdering bool `json:"retainOrdering,omitempty" yaml:"retainOrdering"` + AutoAck bool `json:"autoAck,omitempty" yaml:"autoAck"` + Parallelism int `json:"parallelism,omitempty" yaml:"parallelism"` + Tenant string `json:"tenant,omitempty" yaml:"tenant"` + Namespace string `json:"namespace,omitempty" yaml:"namespace"` + Name string `json:"name,omitempty" yaml:"name"` + ClassName string `json:"className,omitempty" yaml:"className"` - Archive string `json:"archive" yaml:"archive"` - ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` - SourceSubscriptionName string `json:"sourceSubscriptionName" yaml:"sourceSubscriptionName"` - RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` - Inputs []string `json:"inputs" yaml:"inputs"` - TopicToSerdeClassName map[string]string `json:"topicToSerdeClassName" yaml:"topicToSerdeClassName"` - TopicToSchemaType map[string]string `json:"topicToSchemaType" yaml:"topicToSchemaType"` - InputSpecs map[string]ConsumerConfig `json:"inputSpecs" yaml:"inputSpecs"` - Configs map[string]interface{} `json:"configs" yaml:"configs"` + Archive string `json:"archive,omitempty" yaml:"archive"` + ProcessingGuarantees string `json:"processingGuarantees,omitempty" yaml:"processingGuarantees"` + SourceSubscriptionName string `json:"sourceSubscriptionName,omitempty" yaml:"sourceSubscriptionName"` + RuntimeFlags string `json:"runtimeFlags,omitempty" yaml:"runtimeFlags"` + + Inputs []string `json:"inputs,omitempty" yaml:"inputs"` + TopicToSerdeClassName map[string]string `json:"topicToSerdeClassName,omitempty" yaml:"topicToSerdeClassName"` + TopicToSchemaType map[string]string `json:"topicToSchemaType,omitempty" yaml:"topicToSchemaType"` + InputSpecs map[string]ConsumerConfig `json:"inputSpecs,omitempty" yaml:"inputSpecs"` + Configs map[string]interface{} `json:"configs,omitempty" yaml:"configs"` // This is a map of secretName(aka how the secret is going to be // accessed in the function via context) to an object that // encapsulates how the secret is fetched by the underlying // secrets provider. The type of an value here can be found by the // SecretProviderConfigurator.getSecretObjectType() method. - Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` + Secrets map[string]interface{} `json:"secrets,omitempty" yaml:"secrets"` } diff --git a/pulsaradmin/pkg/pulsar/utils/source_config.go b/pulsaradmin/pkg/pulsar/utils/source_config.go index 57b168b959..1e60eaf334 100644 --- a/pulsaradmin/pkg/pulsar/utils/source_config.go +++ b/pulsaradmin/pkg/pulsar/utils/source_config.go @@ -18,28 +18,28 @@ package utils type SourceConfig struct { - Tenant string `json:"tenant" yaml:"tenant"` - Namespace string `json:"namespace" yaml:"namespace"` - Name string `json:"name" yaml:"name"` - ClassName string `json:"className" yaml:"className"` + Tenant string `json:"tenant,omitempty" yaml:"tenant"` + Namespace string `json:"namespace,omitempty" yaml:"namespace"` + Name string `json:"name,omitempty" yaml:"name"` + ClassName string `json:"className,omitempty" yaml:"className"` - TopicName string `json:"topicName" yaml:"topicName"` - SerdeClassName string `json:"serdeClassName" yaml:"serdeClassName"` - SchemaType string `json:"schemaType" yaml:"schemaType"` + TopicName string `json:"topicName,omitempty" yaml:"topicName"` + SerdeClassName string `json:"serdeClassName,omitempty" yaml:"serdeClassName"` + SchemaType string `json:"schemaType,omitempty" yaml:"schemaType"` - Configs map[string]interface{} `json:"configs" yaml:"configs"` + Configs map[string]interface{} `json:"configs,omitempty" yaml:"configs"` // This is a map of secretName(aka how the secret is going to be // accessed in the function via context) to an object that // encapsulates how the secret is fetched by the underlying // secrets provider. The type of an value here can be found by the // SecretProviderConfigurator.getSecretObjectType() method. - Secrets map[string]interface{} `json:"secrets" yaml:"secrets"` + Secrets map[string]interface{} `json:"secrets,omitempty" yaml:"secrets"` - Parallelism int `json:"parallelism" yaml:"parallelism"` - ProcessingGuarantees string `json:"processingGuarantees" yaml:"processingGuarantees"` - Resources *Resources `json:"resources" yaml:"resources"` - Archive string `json:"archive" yaml:"archive"` + Parallelism int `json:"parallelism,omitempty" yaml:"parallelism"` + ProcessingGuarantees string `json:"processingGuarantees,omitempty" yaml:"processingGuarantees"` + Resources *Resources `json:"resources,omitempty" yaml:"resources"` + Archive string `json:"archive,omitempty" yaml:"archive"` // Any flags that you want to pass to the runtime. - RuntimeFlags string `json:"runtimeFlags" yaml:"runtimeFlags"` + RuntimeFlags string `json:"runtimeFlags,omitempty" yaml:"runtimeFlags"` } From 8c6a1230f00857c0032f33313bcafb5d3ebc273b Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 16 Aug 2021 09:40:07 +0800 Subject: [PATCH 198/348] Remove unknown error (streamnative/pulsarctl#422) Signed-off-by: Zixuan Liu Issues: streamnative/pulsarctl#404 ### Changes - Remove `unknownErrorReason` const - Remove return unknown error --- pulsaradmin/pkg/cli/client.go | 19 +++++++++---------- pulsaradmin/pkg/cli/errors.go | 2 -- .../common/algorithm/keypair/keypair.go | 13 ++++++++----- 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index 3319e9e33d..9449207d2b 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -388,24 +388,23 @@ func safeRespClose(resp *http.Response) { // responseError is used to parse a response into a client error func responseError(resp *http.Response) error { - var e Error + e := Error{ + Code: resp.StatusCode, + Reason: resp.Status, + } + body, err := ioutil.ReadAll(resp.Body) if err != nil { e.Reason = err.Error() - e.Code = resp.StatusCode return e } - e.Reason = string(body) err = json.Unmarshal(body, &e) if err != nil { - e.Reason = string(body) - } - - e.Code = resp.StatusCode - - if e.Reason == "" { - e.Reason = unknownErrorReason + if len(body) != 0 { + e.Reason = string(body) + } + return e } return e diff --git a/pulsaradmin/pkg/cli/errors.go b/pulsaradmin/pkg/cli/errors.go index 9a91465a98..e7e2ecbad5 100644 --- a/pulsaradmin/pkg/cli/errors.go +++ b/pulsaradmin/pkg/cli/errors.go @@ -19,8 +19,6 @@ package cli import "fmt" -const unknownErrorReason = "Unknown error" - // Error is a admin error type type Error struct { Reason string `json:"reason"` diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go b/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go index af25265a01..2f6efb34de 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go +++ b/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go @@ -21,15 +21,16 @@ import ( "crypto/ecdsa" "crypto/rsa" "crypto/x509" + "fmt" "github.com/pkg/errors" ) -type KeyType int +type KeyType string const ( - RSA KeyType = iota - ECDSA + RSA KeyType = "RSA" + ECDSA KeyType = "ECDSA" ) // KeyPair saves the ecdsa private key or the rsa private key and provides @@ -69,7 +70,8 @@ func DecodePrivateKey(keyType KeyType, privateKey []byte) (*KeyPair, error) { } return New(ECDSA, key), nil } - return nil, errors.New("unknown error") + + return nil, fmt.Errorf("cannot decode the private key of %s", keyType) } // EncodedPublicKey gets the encoded public key @@ -85,7 +87,8 @@ func (k *KeyPair) EncodedPublicKey() ([]byte, error) { key, _ := k.GetEcdsaPrivateKey() return x509.MarshalPKIXPublicKey(&key.PublicKey) } - return nil, errors.New("unknown error") + + return nil, fmt.Errorf("cannot decode the public key of %s", k.keyType) } // DecodeRSAPublicKey parses the rsa public key. From b17d7a5b8d38526b363d2ae6634df4ea01776423 Mon Sep 17 00:00:00 2001 From: Andy Walker Date: Thu, 19 Aug 2021 20:57:34 -0400 Subject: [PATCH 199/348] add pulsarctl namespace/topic set-publish-rate (streamnative/pulsarctl#428) * add pulsarctl namespace set-publish-rate * satisfy style lint * Add topic-level publishrate --- pulsaradmin/go.mod | 1 - pulsaradmin/pkg/pulsar/namespace.go | 19 ++++++++++++- pulsaradmin/pkg/pulsar/topic.go | 25 ++++++++++++++++ pulsaradmin/pkg/pulsar/utils/data.go | 5 ++++ pulsaradmin/pkg/pulsar/utils/policies.go | 2 ++ pulsaradmin/pkg/pulsar/utils/publish_rate.go | 30 ++++++++++++++++++++ 6 files changed, 80 insertions(+), 2 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/utils/publish_rate.go diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index f24447ad2b..3a5095cc20 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -25,6 +25,5 @@ require ( golang.org/x/net v0.0.0-20210220033124-5f55cee0dc0d // indirect golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93 google.golang.org/appengine v1.6.7 // indirect - google.golang.org/protobuf v1.25.0 // indirect gopkg.in/yaml.v2 v2.3.0 ) diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index 4c8f8600df..5d860652f6 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -253,6 +253,12 @@ type Namespaces interface { // GetDispatchRate returns Message-dispatch-rate (topics under this namespace can dispatch // this many messages per second) GetDispatchRate(namespace utils.NameSpaceName) (utils.DispatchRate, error) + + // SetPublishRate sets the maximum rate or number of messages that producers can publish to topics in this namespace + SetPublishRate(namespace utils.NameSpaceName, pubRate utils.PublishRate) error + + // GetPublishRate gets the maximum rate or number of messages that producer can publish to topics in the namespace + GetPublishRate(namespace utils.NameSpaceName) (utils.PublishRate, error) } type namespaces struct { @@ -454,7 +460,6 @@ func (n *namespaces) SetSchemaAutoUpdateCompatibilityStrategy(namespace utils.Na func (n *namespaces) GetSchemaAutoUpdateCompatibilityStrategy(namespace utils.NameSpaceName) ( utils.SchemaCompatibilityStrategy, error) { - endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "schemaAutoUpdateCompatibilityStrategy") b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) if err != nil { @@ -824,3 +829,15 @@ func (n *namespaces) GetDispatchRate(namespace utils.NameSpaceName) (utils.Dispa err := n.pulsar.Client.Get(endpoint, &rate) return rate, err } + +func (n *namespaces) SetPublishRate(namespace utils.NameSpaceName, pubRate utils.PublishRate) error { + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "publishRate") + return n.pulsar.Client.Post(endpoint, pubRate) +} + +func (n *namespaces) GetPublishRate(namespace utils.NameSpaceName) (utils.PublishRate, error) { + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "publishRate") + var pubRate utils.PublishRate + err := n.pulsar.Client.Get(endpoint, &pubRate) + return pubRate, err +} diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 69b718c1fe..d253c88102 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -171,6 +171,15 @@ type Topics interface { // RemoveDispatchRate Remove message dispatch rate for a topic RemoveDispatchRate(utils.TopicName) error + // GetPublishRate Get message publish rate for a topic + GetPublishRate(utils.TopicName) (*utils.PublishRateData, error) + + // SetPublishRate Set message publish rate for a topic + SetPublishRate(utils.TopicName, utils.PublishRateData) error + + // RemovePublishRate Remove message publish rate for a topic + RemovePublishRate(utils.TopicName) error + // GetDeduplicationStatus Get the deduplication policy for a topic GetDeduplicationStatus(utils.TopicName) (bool, error) @@ -529,6 +538,22 @@ func (t *topics) RemoveDispatchRate(topic utils.TopicName) error { return t.pulsar.Client.Delete(endpoint) } +func (t *topics) GetPublishRate(topic utils.TopicName) (*utils.PublishRateData, error) { + var publishRateData utils.PublishRateData + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "publishRate") + err := t.pulsar.Client.Get(endpoint, &publishRateData) + return &publishRateData, err +} + +func (t *topics) SetPublishRate(topic utils.TopicName, publishRateData utils.PublishRateData) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "publishRate") + return t.pulsar.Client.Post(endpoint, &publishRateData) +} + +func (t *topics) RemovePublishRate(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "publishRate") + return t.pulsar.Client.Delete(endpoint) +} func (t *topics) GetDeduplicationStatus(topic utils.TopicName) (bool, error) { var enabled bool endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "deduplicationEnabled") diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index 103f097333..04655b1515 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -388,3 +388,8 @@ type DispatchRateData struct { RatePeriodInSecond int64 `json:"ratePeriodInSecond"` RelativeToPublishRate bool `json:"relativeToPublishRate"` } + +type PublishRateData struct { + PublishThrottlingRateInMsg int64 `json:"publishThrottlingRateInMsg"` + PublishThrottlingRateInByte int64 `json:"publishThrottlingRateInByte"` +} diff --git a/pulsaradmin/pkg/pulsar/utils/policies.go b/pulsaradmin/pkg/pulsar/utils/policies.go index 0a93afe208..52de13a935 100644 --- a/pulsaradmin/pkg/pulsar/utils/policies.go +++ b/pulsaradmin/pkg/pulsar/utils/policies.go @@ -48,6 +48,7 @@ type Policies struct { TopicDispatchRate map[string]DispatchRate `json:"topicDispatchRate"` SubscriptionDispatchRate map[string]DispatchRate `json:"subscriptionDispatchRate"` ReplicatorDispatchRate map[string]DispatchRate `json:"replicatorDispatchRate"` + PublishMaxMessageRate map[string]PublishRate `json:"publishMaxMessageRate"` ClusterSubscribeRate map[string]SubscribeRate `json:"clusterSubscribeRate"` TopicAutoCreationConfig TopicAutoCreationConfig `json:"autoTopicCreationOverride"` SchemaCompatibilityStrategy SchemaCompatibilityStrategy `json:"schema_auto_update_compatibility_strategy"` @@ -63,6 +64,7 @@ func NewDefaultPolicies() *Policies { TopicDispatchRate: make(map[string]DispatchRate), SubscriptionDispatchRate: make(map[string]DispatchRate), ReplicatorDispatchRate: make(map[string]DispatchRate), + PublishMaxMessageRate: make(map[string]PublishRate), ClusterSubscribeRate: make(map[string]SubscribeRate), LatencyStatsSampleRate: make(map[string]int), MessageTTLInSeconds: 0, diff --git a/pulsaradmin/pkg/pulsar/utils/publish_rate.go b/pulsaradmin/pkg/pulsar/utils/publish_rate.go new file mode 100644 index 0000000000..9dbc2d00d2 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/utils/publish_rate.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 utils + +type PublishRate struct { + PublishThrottlingRateInMsg int `json:"publishThrottlingRateInMsg"` + PublishThrottlingRateInByte int64 `json:"publishThrottlingRateInByte"` +} + +func NewPublishRate() *PublishRate { + return &PublishRate{ + PublishThrottlingRateInMsg: -1, + PublishThrottlingRateInByte: -1, + } +} From 440de491d9ec4d6bb4cc71bb74f57ec254928882 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 23 Aug 2021 09:08:05 +0800 Subject: [PATCH 200/348] feat: add get/remove/set retention policy command for topic (streamnative/pulsarctl#420) background from streamnative/pulsarctl#246, the PR implements the following commands: - `pulsar topics get-retention -a` - Get the retention policy for a topic - `pulsar topics remove-retention ` - Remove the retention policy for a topic - `pulsar topics set-retention --time --size ` - Set the retention policy for a topic --- pulsaradmin/pkg/pulsar/topic.go | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index d253c88102..31924b01e8 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -188,6 +188,15 @@ type Topics interface { // RemoveDeduplicationStatus Remove the deduplication policy for a topic RemoveDeduplicationStatus(utils.TopicName) error + + // GetRetention returns the retention configuration for a topic + GetRetention(utils.TopicName, bool) (*utils.RetentionPolicies, error) + + // RemoveRetention removes the retention configuration on a topic + RemoveRetention(utils.TopicName) error + + // SetRetention sets the retention policy for a topic + SetRetention(utils.TopicName, utils.RetentionPolicies) error } type topics struct { @@ -198,6 +207,9 @@ type topics struct { lookupPath string } +// Check whether the topics struct implements the Topics interface. +var _ Topics = &topics{} + // Topics is used to access the topics endpoints func (c *pulsarClient) Topics() Topics { return &topics{ @@ -569,3 +581,22 @@ func (t *topics) RemoveDeduplicationStatus(topic utils.TopicName) error { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "deduplicationEnabled") return t.pulsar.Client.Delete(endpoint) } + +func (t *topics) GetRetention(topic utils.TopicName, applied bool) (*utils.RetentionPolicies, error) { + var policy utils.RetentionPolicies + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "retention") + _, err := t.pulsar.Client.GetWithQueryParams(endpoint, &policy, map[string]string{ + "applied": strconv.FormatBool(applied), + }, true) + return &policy, err +} + +func (t *topics) RemoveRetention(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "retention") + return t.pulsar.Client.Delete(endpoint) +} + +func (t *topics) SetRetention(topic utils.TopicName, data utils.RetentionPolicies) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "retention") + return t.pulsar.Client.Post(endpoint, data) +} From 87aacec0e91aca38952244cac5e3585c21b37d9b Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 30 Aug 2021 08:59:28 +0800 Subject: [PATCH 201/348] feat: add compaction threshold command for topic (streamnative/pulsarctl#430) ### Changes background from streamnative/pulsarctl#246, the PR implements the following commands: - `pulsarctl topics get-compaction-threshold --applied ` - Get the compaction threshold for a topic - `pulsarctl topics remove-compaction-threshold ` - Remove the compaction threshold for a topic - `pulsarctl topics set-compaction-threshold --threshold ` Set the compaction threshold for a topic --- pulsaradmin/pkg/pulsar/topic.go | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 31924b01e8..2e0ba2fb73 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -197,6 +197,15 @@ type Topics interface { // SetRetention sets the retention policy for a topic SetRetention(utils.TopicName, utils.RetentionPolicies) error + + // Get the compaction threshold for a topic + GetCompactionThreshold(topic utils.TopicName, applied bool) (int64, error) + + // Set the compaction threshold for a topic + SetCompactionThreshold(topic utils.TopicName, threshold int64) error + + // Remove compaction threshold for a topic + RemoveCompactionThreshold(utils.TopicName) error } type topics struct { @@ -600,3 +609,24 @@ func (t *topics) SetRetention(topic utils.TopicName, data utils.RetentionPolicie endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "retention") return t.pulsar.Client.Post(endpoint, data) } + +func (t *topics) GetCompactionThreshold(topic utils.TopicName, applied bool) (int64, error) { + var threshold int64 + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "compactionThreshold") + _, err := t.pulsar.Client.GetWithQueryParams(endpoint, &threshold, map[string]string{ + "applied": strconv.FormatBool(applied), + }, true) + return threshold, err +} + +func (t *topics) SetCompactionThreshold(topic utils.TopicName, threshold int64) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "compactionThreshold") + err := t.pulsar.Client.Post(endpoint, threshold) + return err +} + +func (t *topics) RemoveCompactionThreshold(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "compactionThreshold") + err := t.pulsar.Client.Delete(endpoint) + return err +} From 948541221aa7f6c2b927c127e78cc754b9a64420 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 1 Sep 2021 08:05:34 +0800 Subject: [PATCH 202/348] feat: add backlog quota command for topic (streamnative/pulsarctl#429) Signed-off-by: Zixuan Liu ### Changes background from streamnative/pulsarctl#246, the PR implements the following commands: - `pulsarctl topics get-backlog-quotas -a` - Get the backlog quota policy for a topic - `pulsarctl topics remove-backlog-quota --type ` - Remove a backlog quota policy from a topic - `pulsarctl topics set-backlog-quota --limit-size --limit-time --policy --type ` - Set a backlog quota policy for a topic ### TODO - [x] Add integration tests --- pulsaradmin/pkg/pulsar/topic.go | 43 +++++++++++++++++++ pulsaradmin/pkg/pulsar/utils/backlog_quota.go | 28 ++++++++++-- 2 files changed, 67 insertions(+), 4 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 2e0ba2fb73..82c24b521e 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -19,6 +19,7 @@ package pulsar import ( "fmt" + "net/url" "strconv" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" @@ -206,6 +207,15 @@ type Topics interface { // Remove compaction threshold for a topic RemoveCompactionThreshold(utils.TopicName) error + + // GetBacklogQuotaMap returns backlog quota map for a topic + GetBacklogQuotaMap(topic utils.TopicName, applied bool) (map[utils.BacklogQuotaType]utils.BacklogQuota, error) + + // SetBacklogQuota sets a backlog quota for a topic + SetBacklogQuota(utils.TopicName, utils.BacklogQuota, utils.BacklogQuotaType) error + + // RemoveBacklogQuota removes a backlog quota policy from a topic + RemoveBacklogQuota(utils.TopicName, utils.BacklogQuotaType) error } type topics struct { @@ -630,3 +640,36 @@ func (t *topics) RemoveCompactionThreshold(topic utils.TopicName) error { err := t.pulsar.Client.Delete(endpoint) return err } + +func (t *topics) GetBacklogQuotaMap(topic utils.TopicName, applied bool) (map[utils.BacklogQuotaType]utils.BacklogQuota, + error) { + var backlogQuotaMap map[utils.BacklogQuotaType]utils.BacklogQuota + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "backlogQuotaMap") + + queryParams := map[string]string{"applied": strconv.FormatBool(applied)} + _, err := t.pulsar.Client.GetWithQueryParams(endpoint, &backlogQuotaMap, queryParams, true) + + return backlogQuotaMap, err +} + +func (t *topics) SetBacklogQuota(topic utils.TopicName, backlogQuota utils.BacklogQuota, + backlogQuotaType utils.BacklogQuotaType) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "backlogQuota") + + u, err := url.Parse(endpoint) + if err != nil { + return err + } + q := u.Query() + q.Add("backlogQuotaType", string(backlogQuotaType)) + u.RawQuery = q.Encode() + + return t.pulsar.Client.Post(u.String(), &backlogQuota) +} + +func (t *topics) RemoveBacklogQuota(topic utils.TopicName, backlogQuotaType utils.BacklogQuotaType) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "backlogQuota") + return t.pulsar.Client.DeleteWithQueryParams(endpoint, map[string]string{ + "backlogQuotaType": string(backlogQuotaType), + }) +} diff --git a/pulsaradmin/pkg/pulsar/utils/backlog_quota.go b/pulsaradmin/pkg/pulsar/utils/backlog_quota.go index 2d14f809b6..3b78243bc8 100644 --- a/pulsaradmin/pkg/pulsar/utils/backlog_quota.go +++ b/pulsaradmin/pkg/pulsar/utils/backlog_quota.go @@ -35,10 +35,6 @@ func NewBacklogQuota(limitSize int64, limitTime int64, policy RetentionPolicy) B type RetentionPolicy string -type BacklogQuotaType string - -const DestinationStorage BacklogQuotaType = "destination_storage" - const ( ProducerRequestHold RetentionPolicy = "producer_request_hold" ProducerException RetentionPolicy = "producer_exception" @@ -47,6 +43,8 @@ const ( func ParseRetentionPolicy(str string) (RetentionPolicy, error) { switch str { + case ProducerRequestHold.String(): + return ProducerRequestHold, nil case ProducerException.String(): return ProducerException, nil case ConsumerBacklogEviction.String(): @@ -59,3 +57,25 @@ func ParseRetentionPolicy(str string) (RetentionPolicy, error) { func (s RetentionPolicy) String() string { return string(s) } + +type BacklogQuotaType string + +const ( + DestinationStorage BacklogQuotaType = "destination_storage" + MessageAge BacklogQuotaType = "message_age" +) + +func ParseBacklogQuotaType(str string) (BacklogQuotaType, error) { + switch str { + case DestinationStorage.String(): + return DestinationStorage, nil + case MessageAge.String(): + return MessageAge, nil + default: + return "", errors.Errorf("Invalid backlog quota type: %s", str) + } +} + +func (b BacklogQuotaType) String() string { + return string(b) +} From 228728e005aa2b3151dd785787a5a62888c2f24e Mon Sep 17 00:00:00 2001 From: Jayson Reis Date: Thu, 16 Sep 2021 15:38:40 +0200 Subject: [PATCH 203/348] Add customRuntimeOptions to sinks and functions (streamnative/pulsarctl#449) --- pulsaradmin/pkg/pulsar/utils/function_confg.go | 2 ++ pulsaradmin/pkg/pulsar/utils/sink_config.go | 2 ++ 2 files changed, 4 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/utils/function_confg.go b/pulsaradmin/pkg/pulsar/utils/function_confg.go index d3c873bf19..d082d2dee9 100644 --- a/pulsaradmin/pkg/pulsar/utils/function_confg.go +++ b/pulsaradmin/pkg/pulsar/utils/function_confg.go @@ -68,6 +68,8 @@ type FunctionConfig struct { // A generalized way of specifying inputs InputSpecs map[string]ConsumerConfig `json:"inputSpecs,omitempty" yaml:"inputSpecs"` + CustomRuntimeOptions string `json:"customRuntimeOptions,omitempty" yaml:"customRuntimeOptions"` + // This is a map of secretName(aka how the secret is going to be // accessed in the function via context) to an object that // encapsulates how the secret is fetched by the underlying diff --git a/pulsaradmin/pkg/pulsar/utils/sink_config.go b/pulsaradmin/pkg/pulsar/utils/sink_config.go index aaaf7dc46f..a1878a4b78 100644 --- a/pulsaradmin/pkg/pulsar/utils/sink_config.go +++ b/pulsaradmin/pkg/pulsar/utils/sink_config.go @@ -44,6 +44,8 @@ type SinkConfig struct { InputSpecs map[string]ConsumerConfig `json:"inputSpecs,omitempty" yaml:"inputSpecs"` Configs map[string]interface{} `json:"configs,omitempty" yaml:"configs"` + CustomRuntimeOptions string `json:"customRuntimeOptions,omitempty" yaml:"customRuntimeOptions"` + // This is a map of secretName(aka how the secret is going to be // accessed in the function via context) to an object that // encapsulates how the secret is fetched by the underlying From 2ab48f8e713d6c2238427d8e8f6a93f99c2d63c8 Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Wed, 22 Sep 2021 09:17:03 +0800 Subject: [PATCH 204/348] add packages management service APIs (streamnative/pulsarctl#450) * add packages APIs * fix style * add packages test action * fix style * stash * add tests and fix CI * revert changes * fix unit tests * address comments * address comments * fix upload failure test --- pulsaradmin/pkg/pulsar/admin.go | 1 + pulsaradmin/pkg/pulsar/functions.go | 1 + pulsaradmin/pkg/pulsar/packages.go | 241 ++++++++++++++++++ .../pkg/pulsar/utils/package_metadata.go | 26 ++ pulsaradmin/pkg/pulsar/utils/package_name.go | 116 +++++++++ .../pkg/pulsar/utils/package_name_test.go | 73 ++++++ pulsaradmin/pkg/pulsar/utils/package_type.go | 46 ++++ 7 files changed, 504 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/packages.go create mode 100644 pulsaradmin/pkg/pulsar/utils/package_metadata.go create mode 100644 pulsaradmin/pkg/pulsar/utils/package_name.go create mode 100644 pulsaradmin/pkg/pulsar/utils/package_name_test.go create mode 100644 pulsaradmin/pkg/pulsar/utils/package_type.go diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 3f61af2f65..5a2ffe807c 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -51,6 +51,7 @@ type Client interface { ResourceQuotas() ResourceQuotas FunctionsWorker() FunctionsWorker Token() Token + Packages() Packages } type pulsarClient struct { diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index b9d10c5003..77807fd8b7 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -663,6 +663,7 @@ func (f *functions) Upload(sourceFile, path string) error { if err != nil { return err } + defer file.Close() endpoint := f.pulsar.endpoint(f.basePath, "upload") var b bytes.Buffer w := multipart.NewWriter(&b) diff --git a/pulsaradmin/pkg/pulsar/packages.go b/pulsaradmin/pkg/pulsar/packages.go new file mode 100644 index 0000000000..eb226dd9bb --- /dev/null +++ b/pulsaradmin/pkg/pulsar/packages.go @@ -0,0 +1,241 @@ +// 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 pulsar + +import ( + "bytes" + "encoding/json" + "fmt" + "io" + "mime/multipart" + "net/textproto" + "os" + "path/filepath" + "strings" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" +) + +// Packages is admin interface for functions management +type Packages interface { + // Download Function/Connector Package + // @param destinationFile + // file where data should be downloaded to + // @param packageURL + // the package URL + Download(packageURL, destinationFile string) error + + // Upload Function/Connector Package + // @param filePath + // file where data should be uploaded to + // @param packageURL + // type://tenant/namespace/packageName@version + // @param description + // descriptions of a package + // @param contact + // contact information of a package + // @param properties + // external infromations of a package + + Upload(packageURL, filePath, description, contact string, properties map[string]string) error + + // List all the packages with the given type in a namespace + List(typeName, namespace string) ([]string, error) + + // ListVersions list all the versions of a package + ListVersions(packageURL string) ([]string, error) + + // Delete the specified package + Delete(packageURL string) error + + // GetMetadata get a package metadata information + GetMetadata(packageURL string) (utils.PackageMetadata, error) + + // UpdateMetadata update a package metadata information + UpdateMetadata(packageURL, description, contact string, properties map[string]string) error +} + +type packages struct { + pulsar *pulsarClient + basePath string +} + +func (p *packages) createStringFromField(w *multipart.Writer, value string) (io.Writer, error) { + h := make(textproto.MIMEHeader) + h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="%s" `, value)) + h.Set("Content-Type", "application/json") + return w.CreatePart(h) +} + +// Packages is used to access the functions endpoints +func (c *pulsarClient) Packages() Packages { + return &packages{ + pulsar: c, + basePath: "/packages", + } +} + +func (p packages) Download(packageURL, destinationFile string) error { + packageName, err := utils.GetPackageName(packageURL) + if err != nil { + return err + } + endpoint := p.pulsar.endpoint(p.basePath, string(packageName.GetType()), packageName.GetTenant(), + packageName.GetNamespace(), packageName.GetName(), packageName.GetVersion()) + _, err = os.Open(destinationFile) + if err != nil { + if !os.IsNotExist(err) { + return fmt.Errorf("file %s already exists, please delete "+ + "the file first or change the file name", destinationFile) + } + } + file, err := os.Create(destinationFile) + if err != nil { + return err + } + + _, err = p.pulsar.Client.GetWithOptions(endpoint, nil, nil, false, file) + if err != nil { + return err + } + return nil +} + +func (p packages) Upload(packageURL, filePath, description, contact string, properties map[string]string) error { + if strings.TrimSpace(filePath) == "" && strings.TrimSpace(packageURL) == "" { + return fmt.Errorf("packageURL or file path is empty") + } + packageName, err := utils.GetPackageName(packageURL) + if err != nil { + return err + } + endpoint := p.pulsar.endpoint(p.basePath, string(packageName.GetType()), packageName.GetTenant(), + packageName.GetNamespace(), packageName.GetName(), packageName.GetVersion()) + metadata := utils.PackageMetadata{ + Description: description, + Contact: contact, + Properties: properties, + } + // buffer to store our request as bytes + bodyBuf := bytes.NewBufferString("") + + multiPartWriter := multipart.NewWriter(bodyBuf) + + metadataJSON, err := json.Marshal(metadata) + if err != nil { + return err + } + + stringWriter, err := p.createStringFromField(multiPartWriter, "metadata") + if err != nil { + return err + } + + _, err = stringWriter.Write(metadataJSON) + if err != nil { + return err + } + + file, err := os.Open(filePath) + if err != nil { + return err + } + defer file.Close() + + part, err := multiPartWriter.CreateFormFile("file", filepath.Base(file.Name())) + + if err != nil { + return err + } + + // copy the actual file content to the filed's writer + _, err = io.Copy(part, file) + if err != nil { + return err + } + + if err = multiPartWriter.Close(); err != nil { + return err + } + + contentType := multiPartWriter.FormDataContentType() + err = p.pulsar.Client.PostWithMultiPart(endpoint, nil, bodyBuf, contentType) + if err != nil { + return err + } + + return nil +} + +func (p packages) List(typeName, namespace string) ([]string, error) { + var packageList []string + endpoint := p.pulsar.endpoint(p.basePath, typeName, namespace) + err := p.pulsar.Client.Get(endpoint, &packageList) + return packageList, err +} + +func (p packages) ListVersions(packageURL string) ([]string, error) { + var versionList []string + packageName, err := utils.GetPackageName(packageURL) + if err != nil { + return versionList, err + } + endpoint := p.pulsar.endpoint(p.basePath, string(packageName.GetType()), packageName.GetTenant(), + packageName.GetNamespace(), packageName.GetName()) + err = p.pulsar.Client.Get(endpoint, &versionList) + return versionList, err +} + +func (p packages) Delete(packageURL string) error { + packageName, err := utils.GetPackageName(packageURL) + if err != nil { + return err + } + endpoint := p.pulsar.endpoint(p.basePath, string(packageName.GetType()), packageName.GetTenant(), + packageName.GetNamespace(), packageName.GetName(), packageName.GetVersion()) + + return p.pulsar.Client.Delete(endpoint) +} + +func (p packages) GetMetadata(packageURL string) (utils.PackageMetadata, error) { + var metadata utils.PackageMetadata + packageName, err := utils.GetPackageName(packageURL) + if err != nil { + return metadata, err + } + endpoint := p.pulsar.endpoint(p.basePath, string(packageName.GetType()), packageName.GetTenant(), + packageName.GetNamespace(), packageName.GetName(), packageName.GetVersion(), "metadata") + err = p.pulsar.Client.Get(endpoint, &metadata) + return metadata, err +} + +func (p packages) UpdateMetadata(packageURL, description, contact string, properties map[string]string) error { + metadata := utils.PackageMetadata{ + Description: description, + Contact: contact, + Properties: properties, + } + packageName, err := utils.GetPackageName(packageURL) + if err != nil { + return err + } + endpoint := p.pulsar.endpoint(p.basePath, string(packageName.GetType()), packageName.GetTenant(), + packageName.GetNamespace(), packageName.GetName(), packageName.GetVersion(), "metadata") + + return p.pulsar.Client.Put(endpoint, &metadata) +} diff --git a/pulsaradmin/pkg/pulsar/utils/package_metadata.go b/pulsaradmin/pkg/pulsar/utils/package_metadata.go new file mode 100644 index 0000000000..860afd9c8e --- /dev/null +++ b/pulsaradmin/pkg/pulsar/utils/package_metadata.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 utils + +type PackageMetadata struct { + Description string `json:"description,omitempty" yaml:"description"` + Contact string `json:"contact,omitempty" yaml:"contact"` + CreateTime int64 `json:"createTime,omitempty" yaml:"createTime"` + ModificationTime int64 `json:"modificationTime,omitempty" yaml:"modificationTime"` + Properties map[string]string `json:"properties,omitempty" yaml:"properties"` +} diff --git a/pulsaradmin/pkg/pulsar/utils/package_name.go b/pulsaradmin/pkg/pulsar/utils/package_name.go new file mode 100644 index 0000000000..afc646d687 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/utils/package_name.go @@ -0,0 +1,116 @@ +// 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 utils + +import ( + "fmt" + "strings" + + "github.com/pkg/errors" +) + +type PackageName struct { + packageType PackageType + namespace string + tenant string + name string + version string + completePackageName string + completeName string +} + +func invalidPackageNameError(completeName string) error { + return errors.Errorf("Invalid package name '%s', it should be "+ + "in the format of type://tenant/namespace/name@version", completeName) +} + +func GetPackageNameWithComponents(packageType PackageType, + tenant, namespace, name, version string) (*PackageName, error) { + return GetPackageName(fmt.Sprintf("%s://%s/%s/%s@%s", packageType, tenant, namespace, name, version)) +} + +func GetPackageName(completeName string) (*PackageName, error) { + var packageName PackageName + var err error + if !strings.Contains(completeName, "://") { + return nil, invalidPackageNameError(completeName) + } + parts := strings.Split(completeName, "://") + if len(parts) != 2 { + return nil, invalidPackageNameError(completeName) + } + packageName.packageType, err = parsePackageType(parts[0]) + if err != nil { + return nil, err + } + rest := parts[1] + if !strings.Contains(rest, "@") { + // if the package name does not contains '@', that means user does not set the version of package. + // We will set the version to latest. + rest += "@" + } + parts = strings.Split(rest, "@") + if len(parts) != 2 { + return nil, invalidPackageNameError(completeName) + } + partsWithoutVersion := strings.Split(parts[0], "/") + if len(partsWithoutVersion) != 3 { + return nil, invalidPackageNameError(completeName) + } + packageName.tenant = partsWithoutVersion[0] + packageName.namespace = partsWithoutVersion[1] + packageName.name = partsWithoutVersion[2] + packageName.version = "latest" + if parts[1] != "" { + packageName.version = parts[1] + } + packageName.completeName = fmt.Sprintf("%s/%s/%s", + packageName.tenant, packageName.namespace, packageName.name) + packageName.completePackageName = fmt.Sprintf("%s://%s/%s/%s@%s", + packageName.packageType, packageName.tenant, packageName.namespace, packageName.name, packageName.version) + + return &packageName, nil +} + +func (p *PackageName) String() string { + return p.completePackageName +} + +func (p *PackageName) GetType() PackageType { + return p.packageType +} + +func (p *PackageName) GetTenant() string { + return p.tenant +} + +func (p *PackageName) GetNamespace() string { + return p.namespace +} + +func (p *PackageName) GetName() string { + return p.name +} + +func (p *PackageName) GetVersion() string { + return p.version +} + +func (p *PackageName) GetCompleteName() string { + return p.completeName +} diff --git a/pulsaradmin/pkg/pulsar/utils/package_name_test.go b/pulsaradmin/pkg/pulsar/utils/package_name_test.go new file mode 100644 index 0000000000..8e6edd52d8 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/utils/package_name_test.go @@ -0,0 +1,73 @@ +// 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 utils + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestGetPackageName(t *testing.T) { + success, err := GetPackageName("function://f-tenant/f-ns/f-name@f-version") + assert.Nil(t, err) + assert.Equal(t, "function://f-tenant/f-ns/f-name@f-version", success.String()) + + success, err = GetPackageName("function://f-tenant/f-ns/f-name") + assert.Nil(t, err) + assert.Equal(t, "function://f-tenant/f-ns/f-name@latest", success.String()) + + success, err = GetPackageName("sink://s-tenant/s-ns/s-name@s-version") + assert.Nil(t, err) + assert.Equal(t, "sink://s-tenant/s-ns/s-name@s-version", success.String()) + + success, err = GetPackageName("sink://s-tenant/s-ns/s-name") + assert.Nil(t, err) + assert.Equal(t, "sink://s-tenant/s-ns/s-name@latest", success.String()) + + success, err = GetPackageName("source://s-tenant/s-ns/s-name@s-version") + assert.Nil(t, err) + assert.Equal(t, "source://s-tenant/s-ns/s-name@s-version", success.String()) + + success, err = GetPackageName("source://s-tenant/s-ns/s-name") + assert.Nil(t, err) + assert.Equal(t, "source://s-tenant/s-ns/s-name@latest", success.String()) + + fail, err := GetPackageName("function:///public/default/test-error@v1") + assert.NotNil(t, err) + assert.Equal(t, "Invalid package name 'function:///public/default/test-error@v1', it should be in the "+ + "format of type://tenant/namespace/name@version", err.Error()) + assert.Nil(t, fail) + + fail, err = GetPackageNameWithComponents("functions", "public", "default", "test-error", "v1") + assert.NotNil(t, err) + assert.Equal(t, "Invalid package type 'functions', it should be function, sink, or source", err.Error()) + assert.Nil(t, fail) + + fail, err = GetPackageNameWithComponents("function", "public/default", "default", "test-error", "v1") + assert.NotNil(t, err) + assert.Equal(t, "Invalid package name 'function://public/default/default/test-error@v1', it should be in the "+ + "format of type://tenant/namespace/name@version", err.Error()) + assert.Nil(t, fail) + + fail, err = GetPackageName("function://public/default/test-error-version/v2") + assert.NotNil(t, err) + assert.Equal(t, "Invalid package name 'function://public/default/test-error-version/v2', it should be in the "+ + "format of type://tenant/namespace/name@version", err.Error()) + assert.Nil(t, fail) +} diff --git a/pulsaradmin/pkg/pulsar/utils/package_type.go b/pulsaradmin/pkg/pulsar/utils/package_type.go new file mode 100644 index 0000000000..70f455b253 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/utils/package_type.go @@ -0,0 +1,46 @@ +// 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 utils + +import "github.com/pkg/errors" + +type PackageType string + +const ( + PackageTypeFunction PackageType = "function" + PackageTypeSink PackageType = "sink" + PackageTypeSource PackageType = "source" +) + +func parsePackageType(packageTypeName string) (PackageType, error) { + switch packageTypeName { + case PackageTypeFunction.String(): + return PackageTypeFunction, nil + case PackageTypeSink.String(): + return PackageTypeSink, nil + case PackageTypeSource.String(): + return PackageTypeSource, nil + default: + return "", errors.Errorf("Invalid package type '%s', it should be "+ + "function, sink, or source", packageTypeName) + } +} + +func (p PackageType) String() string { + return string(p) +} From a9d3a3dd286107c8610991a670d477826a948cda Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Fri, 24 Sep 2021 11:53:25 +0800 Subject: [PATCH 205/348] fix packages cmds (streamnative/pulsarctl#458) --- pulsaradmin/pkg/pulsar/packages.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/packages.go b/pulsaradmin/pkg/pulsar/packages.go index eb226dd9bb..e0029f6964 100644 --- a/pulsaradmin/pkg/pulsar/packages.go +++ b/pulsaradmin/pkg/pulsar/packages.go @@ -51,7 +51,6 @@ type Packages interface { // contact information of a package // @param properties // external infromations of a package - Upload(packageURL, filePath, description, contact string, properties map[string]string) error // List all the packages with the given type in a namespace From 968ad573769d1aeeb764b899d61bad440a863909 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 28 Sep 2021 15:56:34 +0800 Subject: [PATCH 206/348] feat: add inactive topic policies command for topic (streamnative/pulsarctl#444) Signed-off-by: Zixuan Liu ### Changes background from streamnative/pulsarctl#246, the PR implements the following commands: - `pulsarctl topics get-inactive-topic-policies --applied ` - `pulsarctl topics remove-inactive-topic-policies ` - `pulsarctl topics set-inactive-topic-policies --enable-delete-while-inactive --max-inactive-duration --delete-mode ` --- pulsaradmin/pkg/pulsar/topic.go | 28 +++++++++ .../pulsar/utils/inactive_topic_policies.go | 59 +++++++++++++++++++ 2 files changed, 87 insertions(+) create mode 100644 pulsaradmin/pkg/pulsar/utils/inactive_topic_policies.go diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 82c24b521e..50061fb075 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -216,6 +216,15 @@ type Topics interface { // RemoveBacklogQuota removes a backlog quota policy from a topic RemoveBacklogQuota(utils.TopicName, utils.BacklogQuotaType) error + + // GetInactiveTopicPolicies gets the inactive topic policies on a topic + GetInactiveTopicPolicies(topic utils.TopicName, applied bool) (utils.InactiveTopicPolicies, error) + + // RemoveInactiveTopicPolicies removes inactive topic policies from a topic + RemoveInactiveTopicPolicies(utils.TopicName) error + + // SetInactiveTopicPolicies sets the inactive topic policies on a topic + SetInactiveTopicPolicies(topic utils.TopicName, data utils.InactiveTopicPolicies) error } type topics struct { @@ -673,3 +682,22 @@ func (t *topics) RemoveBacklogQuota(topic utils.TopicName, backlogQuotaType util "backlogQuotaType": string(backlogQuotaType), }) } + +func (t *topics) GetInactiveTopicPolicies(topic utils.TopicName, applied bool) (utils.InactiveTopicPolicies, error) { + var out utils.InactiveTopicPolicies + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "inactiveTopicPolicies") + _, err := t.pulsar.Client.GetWithQueryParams(endpoint, &out, map[string]string{ + "applied": strconv.FormatBool(applied), + }, true) + return out, err +} + +func (t *topics) RemoveInactiveTopicPolicies(topic utils.TopicName) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "inactiveTopicPolicies") + return t.pulsar.Client.Delete(endpoint) +} + +func (t *topics) SetInactiveTopicPolicies(topic utils.TopicName, data utils.InactiveTopicPolicies) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "inactiveTopicPolicies") + return t.pulsar.Client.Post(endpoint, data) +} diff --git a/pulsaradmin/pkg/pulsar/utils/inactive_topic_policies.go b/pulsaradmin/pkg/pulsar/utils/inactive_topic_policies.go new file mode 100644 index 0000000000..05f81b6645 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/utils/inactive_topic_policies.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 utils + +import "github.com/pkg/errors" + +type InactiveTopicDeleteMode string + +const ( + // The topic can be deleted when no subscriptions and no active producers. + DeleteWhenNoSubscriptions InactiveTopicDeleteMode = "delete_when_no_subscriptions" + // The topic can be deleted when all subscriptions catchup and no active producers/consumers. + DeleteWhenSubscriptionsCaughtUp InactiveTopicDeleteMode = "delete_when_subscriptions_caught_up" +) + +func (i InactiveTopicDeleteMode) String() string { + return string(i) +} + +func ParseInactiveTopicDeleteMode(str string) (InactiveTopicDeleteMode, error) { + switch str { + case DeleteWhenNoSubscriptions.String(): + return DeleteWhenNoSubscriptions, nil + case DeleteWhenSubscriptionsCaughtUp.String(): + return DeleteWhenSubscriptionsCaughtUp, nil + default: + return "", errors.Errorf("cannot parse %s to InactiveTopicDeleteMode type", str) + } +} + +type InactiveTopicPolicies struct { + InactiveTopicDeleteMode *InactiveTopicDeleteMode `json:"inactiveTopicDeleteMode"` + MaxInactiveDurationSeconds int `json:"maxInactiveDurationSeconds"` + DeleteWhileInactive bool `json:"deleteWhileInactive"` +} + +func NewInactiveTopicPolicies(inactiveTopicDeleteMode *InactiveTopicDeleteMode, maxInactiveDurationSeconds int, + deleteWhileInactive bool) InactiveTopicPolicies { + return InactiveTopicPolicies{ + InactiveTopicDeleteMode: inactiveTopicDeleteMode, + MaxInactiveDurationSeconds: maxInactiveDurationSeconds, + DeleteWhileInactive: deleteWhileInactive, + } +} From 2fbf81ab5f70e6a4e2854f9f9e3b2ffdc639ae9f Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 28 Sep 2021 16:10:30 +0800 Subject: [PATCH 207/348] fix: fix checks the packages url and file path of the upload package (streamnative/pulsarctl#460) Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/pulsar/packages.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/packages.go b/pulsaradmin/pkg/pulsar/packages.go index e0029f6964..ef9c773f97 100644 --- a/pulsaradmin/pkg/pulsar/packages.go +++ b/pulsaradmin/pkg/pulsar/packages.go @@ -28,6 +28,8 @@ import ( "path/filepath" "strings" + "github.com/pkg/errors" + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" ) @@ -116,8 +118,11 @@ func (p packages) Download(packageURL, destinationFile string) error { } func (p packages) Upload(packageURL, filePath, description, contact string, properties map[string]string) error { - if strings.TrimSpace(filePath) == "" && strings.TrimSpace(packageURL) == "" { - return fmt.Errorf("packageURL or file path is empty") + if strings.TrimSpace(filePath) == "" { + return errors.New("file path is empty") + } + if strings.TrimSpace(packageURL) == "" { + return errors.New("package URL is empty") } packageName, err := utils.GetPackageName(packageURL) if err != nil { From eaa13e6362198b658a3ec0531541a2f7a7e801c9 Mon Sep 17 00:00:00 2001 From: Eron Wright Date: Thu, 7 Oct 2021 11:13:57 -0700 Subject: [PATCH 208/348] OAuth 2.0 Usability Improvements (+Azure AD) (streamnative/pulsarctl#465) * Prefer command-line flags over configuration settings. * Use context for OAuth2 flags. * Improve support for Azure AD (scopes, verification url) * Update go.mod --- pulsaradmin/go.mod | 12 ++++++++++-- pulsaradmin/pkg/auth/auth_provider.go | 6 ++---- pulsaradmin/pkg/auth/oauth2.go | 15 ++++++++------- pulsaradmin/pkg/pulsar/common/config.go | 1 + 4 files changed, 21 insertions(+), 13 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index 3a5095cc20..8d708bd2b7 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -4,13 +4,17 @@ go 1.13 require ( github.com/99designs/keyring v1.1.6 - github.com/apache/pulsar-client-go/oauth2 v0.0.0-20210220083636-af91e9ca0ee2 + github.com/apache/pulsar-client-go/oauth2 v0.0.0-20211006154457-742f1b107403 github.com/dgrijalva/jwt-go v3.2.0+incompatible github.com/docker/go-connections v0.4.0 github.com/fatih/color v1.7.0 github.com/ghodss/yaml v1.0.0 + github.com/go-sql-driver/mysql v1.5.0 // indirect + github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.4.3 + github.com/gorilla/mux v1.7.4 // indirect github.com/imdario/mergo v0.3.8 + github.com/kr/pretty v0.2.0 // indirect github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b github.com/magiconair/properties v1.8.0 @@ -18,12 +22,16 @@ require ( github.com/mattn/go-runewidth v0.0.4 // indirect github.com/olekukonko/tablewriter v0.0.1 github.com/pkg/errors v0.9.1 + github.com/sirupsen/logrus v1.4.2 // indirect github.com/spf13/cobra v0.0.5 github.com/spf13/pflag v1.0.5 - github.com/stretchr/testify v1.4.0 + github.com/stretchr/testify v1.5.1 github.com/testcontainers/testcontainers-go v0.0.10 golang.org/x/net v0.0.0-20210220033124-5f55cee0dc0d // indirect golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93 google.golang.org/appengine v1.6.7 // indirect + gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 // indirect gopkg.in/yaml.v2 v2.3.0 ) + +replace github.com/apache/pulsar-client-go/oauth2 => github.com/apache/pulsar-client-go/oauth2 v0.0.0-20211006154457-742f1b107403 diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index 7c2e358583..c90e834c3d 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -60,11 +60,9 @@ func GetAuthProvider(config *common.Config) (*Provider, error) { provider, err = NewAuthenticationToken(config.Token, defaultTransport) case len(config.TokenFile) > 0: provider, err = NewAuthenticationTokenFromFile(config.TokenFile, defaultTransport) - case len(config.IssuerEndpoint) > 0 || len(config.KeyFile) > 0: + case len(config.IssuerEndpoint) > 0 || len(config.ClientID) > 0 || len(config.Audience) > 0 || len(config.Scope) > 0: provider, err = NewAuthenticationOAuth2WithParams( - config.IssuerEndpoint, - config.ClientID, - config.Audience, defaultTransport) + config.IssuerEndpoint, config.ClientID, config.Audience, config.Scope, defaultTransport) } } return &provider, err diff --git a/pulsaradmin/pkg/auth/oauth2.go b/pulsaradmin/pkg/auth/oauth2.go index a15a555322..1e50adb9e2 100644 --- a/pulsaradmin/pkg/auth/oauth2.go +++ b/pulsaradmin/pkg/auth/oauth2.go @@ -21,6 +21,8 @@ import ( "net/http" "path/filepath" + "github.com/pkg/errors" + "github.com/99designs/keyring" "github.com/apache/pulsar-client-go/oauth2" "github.com/apache/pulsar-client-go/oauth2/cache" @@ -30,11 +32,6 @@ import ( xoauth2 "golang.org/x/oauth2" ) -const ( - TypeClientCredential = "client_credentials" - TypeDeviceCode = "device_code" -) - type OAuth2Provider struct { clock clock2.RealClock issuer oauth2.Issuer @@ -89,13 +86,14 @@ func NewAuthenticationOAuth2WithDefaultFlow(issuer oauth2.Issuer, keyFile string } func NewAuthenticationOAuth2WithParams( - issueEndpoint, + issuerEndpoint, clientID, audience string, + scope string, transport http.RoundTripper) (*OAuth2Provider, error) { issuer := oauth2.Issuer{ - IssuerEndpoint: issueEndpoint, + IssuerEndpoint: issuerEndpoint, ClientID: clientID, Audience: audience, } @@ -123,6 +121,9 @@ func NewAuthenticationOAuth2WithParams( func (o *OAuth2Provider) loadGrant() error { grant, err := o.store.LoadGrant(o.issuer.Audience) if err != nil { + if err == store.ErrNoAuthenticationData { + return errors.New("oauth2 login required") + } return err } return o.initCache(grant) diff --git a/pulsaradmin/pkg/pulsar/common/config.go b/pulsaradmin/pkg/pulsar/common/config.go index 09a4c0c039..317e4bd34e 100644 --- a/pulsaradmin/pkg/pulsar/common/config.go +++ b/pulsaradmin/pkg/pulsar/common/config.go @@ -48,4 +48,5 @@ type Config struct { ClientID string Audience string KeyFile string + Scope string } From 5bbd7b6c993f7dcda9d50e76cd284373683531e2 Mon Sep 17 00:00:00 2001 From: Jayson Reis Date: Sat, 9 Oct 2021 06:43:13 +0200 Subject: [PATCH 209/348] Add support for sourceSubscriptionName to sinks (streamnative/pulsarctl#469) --- pulsaradmin/pkg/pulsar/utils/data.go | 1 + pulsaradmin/pkg/pulsar/utils/sink_config.go | 9 +++++---- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index 04655b1515..91a32cf3f0 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -134,6 +134,7 @@ type SinkData struct { Inputs string `json:"inputs,omitempty"` TopicsPattern string `json:"topicsPattern,omitempty"` SubsName string `json:"subsName,omitempty"` + SubsPosition string `json:"subsPosition,omitempty"` CustomSerdeInputString string `json:"customSerdeInputString,omitempty"` CustomSchemaInputString string `json:"customSchemaInputString,omitempty"` ProcessingGuarantees string `json:"processingGuarantees,omitempty"` diff --git a/pulsaradmin/pkg/pulsar/utils/sink_config.go b/pulsaradmin/pkg/pulsar/utils/sink_config.go index a1878a4b78..377fa5b629 100644 --- a/pulsaradmin/pkg/pulsar/utils/sink_config.go +++ b/pulsaradmin/pkg/pulsar/utils/sink_config.go @@ -33,10 +33,11 @@ type SinkConfig struct { Name string `json:"name,omitempty" yaml:"name"` ClassName string `json:"className,omitempty" yaml:"className"` - Archive string `json:"archive,omitempty" yaml:"archive"` - ProcessingGuarantees string `json:"processingGuarantees,omitempty" yaml:"processingGuarantees"` - SourceSubscriptionName string `json:"sourceSubscriptionName,omitempty" yaml:"sourceSubscriptionName"` - RuntimeFlags string `json:"runtimeFlags,omitempty" yaml:"runtimeFlags"` + Archive string `json:"archive,omitempty" yaml:"archive"` + ProcessingGuarantees string `json:"processingGuarantees,omitempty" yaml:"processingGuarantees"` + SourceSubscriptionName string `json:"sourceSubscriptionName,omitempty" yaml:"sourceSubscriptionName"` + SourceSubscriptionPosition string `json:"sourceSubscriptionPosition,omitempty" yaml:"sourceSubscriptionPosition"` + RuntimeFlags string `json:"runtimeFlags,omitempty" yaml:"runtimeFlags"` Inputs []string `json:"inputs,omitempty" yaml:"inputs"` TopicToSerdeClassName map[string]string `json:"topicToSerdeClassName,omitempty" yaml:"topicToSerdeClassName"` From 887c85ca89aff93f807e3841b762471faf8e9f03 Mon Sep 17 00:00:00 2001 From: Marvin Cai Date: Sat, 23 Oct 2021 13:26:40 +0800 Subject: [PATCH 210/348] Support BacklogQuotaType when setting namespace backlog. (streamnative/pulsarctl#480) * Support BacklogQuotaType when setting namespace backlog. * Fix post with params. * Debug test. * debug print * Update. * Update. * Revert. --- pulsaradmin/pkg/cli/client.go | 5 ++++- pulsaradmin/pkg/pulsar/namespace.go | 9 ++++++--- pulsaradmin/pkg/pulsar/topic.go | 16 ++++------------ pulsaradmin/pkg/pulsar/utils/backlog_quota.go | 2 ++ pulsaradmin/pkg/pulsar/utils/data.go | 1 + 5 files changed, 17 insertions(+), 16 deletions(-) diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index 9449207d2b..e3775f5ed8 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -280,11 +280,14 @@ func (c *Client) PostWithMultiPart(endpoint string, in interface{}, body io.Read return nil } -func (c *Client) PostWithQueryParams(endpoint string, params map[string]string) error { +func (c *Client) PostWithQueryParams(endpoint string, in interface{}, params map[string]string) error { req, err := c.newRequest(http.MethodPost, endpoint) if err != nil { return err } + if in != nil { + req.obj = in + } if params != nil { query := req.url.Query() for k, v := range params { diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index 5d860652f6..bfe07e67ee 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -71,7 +71,7 @@ type Namespaces interface { GetBacklogQuotaMap(namespace string) (map[utils.BacklogQuotaType]utils.BacklogQuota, error) // SetBacklogQuota sets a backlog quota for all the topics on a namespace - SetBacklogQuota(namespace string, backlogQuota utils.BacklogQuota) error + SetBacklogQuota(namespace string, backlogQuota utils.BacklogQuota, backlogQuotaType utils.BacklogQuotaType) error // RemoveBacklogQuota removes a backlog quota policy from a namespace RemoveBacklogQuota(namespace string) error @@ -407,13 +407,16 @@ func (n *namespaces) GetBacklogQuotaMap(namespace string) (map[utils.BacklogQuot return backlogQuotaMap, err } -func (n *namespaces) SetBacklogQuota(namespace string, backlogQuota utils.BacklogQuota) error { +func (n *namespaces) SetBacklogQuota(namespace string, backlogQuota utils.BacklogQuota, + backlogQuotaType utils.BacklogQuotaType) error { nsName, err := utils.GetNamespaceName(namespace) if err != nil { return err } endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "backlogQuota") - return n.pulsar.Client.Post(endpoint, &backlogQuota) + params := make(map[string]string) + params["backlogQuotaType"] = string(backlogQuotaType) + return n.pulsar.Client.PostWithQueryParams(endpoint, &backlogQuota, params) } func (n *namespaces) RemoveBacklogQuota(namespace string) error { diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 50061fb075..f949d1a0cb 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -19,7 +19,6 @@ package pulsar import ( "fmt" - "net/url" "strconv" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" @@ -443,7 +442,7 @@ func (t *topics) SetMessageTTL(topic utils.TopicName, messageTTL int) error { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "messageTTL") var params = make(map[string]string) params["messageTTL"] = strconv.Itoa(messageTTL) - err := t.pulsar.Client.PostWithQueryParams(endpoint, params) + err := t.pulsar.Client.PostWithQueryParams(endpoint, nil, params) return err } @@ -664,16 +663,9 @@ func (t *topics) GetBacklogQuotaMap(topic utils.TopicName, applied bool) (map[ut func (t *topics) SetBacklogQuota(topic utils.TopicName, backlogQuota utils.BacklogQuota, backlogQuotaType utils.BacklogQuotaType) error { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "backlogQuota") - - u, err := url.Parse(endpoint) - if err != nil { - return err - } - q := u.Query() - q.Add("backlogQuotaType", string(backlogQuotaType)) - u.RawQuery = q.Encode() - - return t.pulsar.Client.Post(u.String(), &backlogQuota) + params := make(map[string]string) + params["backlogQuotaType"] = string(backlogQuotaType) + return t.pulsar.Client.PostWithQueryParams(endpoint, &backlogQuota, params) } func (t *topics) RemoveBacklogQuota(topic utils.TopicName, backlogQuotaType utils.BacklogQuotaType) error { diff --git a/pulsaradmin/pkg/pulsar/utils/backlog_quota.go b/pulsaradmin/pkg/pulsar/utils/backlog_quota.go index 3b78243bc8..1930cad8ce 100644 --- a/pulsaradmin/pkg/pulsar/utils/backlog_quota.go +++ b/pulsaradmin/pkg/pulsar/utils/backlog_quota.go @@ -67,6 +67,8 @@ const ( func ParseBacklogQuotaType(str string) (BacklogQuotaType, error) { switch str { + case "": + fallthrough case DestinationStorage.String(): return DestinationStorage, nil case MessageAge.String(): diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index 91a32cf3f0..4941456dd8 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -174,6 +174,7 @@ type NamespacesData struct { LimitStr string `json:"limitStr"` LimitTime int64 `json:"limitTime"` PolicyStr string `json:"policyStr"` + BacklogQuotaType string `json:"backlogQuotaType"` AntiAffinityGroup string `json:"antiAffinityGroup"` Tenant string `json:"tenant"` Cluster string `json:"cluster"` From 30f982c8469e1a5649e06281390ea7ac17feade0 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 26 Oct 2021 22:35:24 +0800 Subject: [PATCH 211/348] fix: improve parse response avoid parse the empty body (streamnative/pulsarctl#491) Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/pulsar/namespace.go | 56 +++++++++++------------------ 1 file changed, 21 insertions(+), 35 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index bfe07e67ee..efeee0558f 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -447,12 +447,10 @@ func (n *namespaces) SetSchemaValidationEnforced(namespace utils.NameSpaceName, } func (n *namespaces) GetSchemaValidationEnforced(namespace utils.NameSpaceName) (bool, error) { + var result bool endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "schemaValidationEnforced") - r, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) - if err != nil { - return false, err - } - return strconv.ParseBool(string(r)) + err := n.pulsar.Client.Get(endpoint, &result) + return result, err } func (n *namespaces) SetSchemaAutoUpdateCompatibilityStrategy(namespace utils.NameSpaceName, @@ -486,12 +484,10 @@ func (n *namespaces) SetOffloadDeleteLag(namespace utils.NameSpaceName, timeMs i } func (n *namespaces) GetOffloadDeleteLag(namespace utils.NameSpaceName) (int64, error) { + var result int64 endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "offloadDeletionLagMs") - b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) - if err != nil { - return -1, err - } - return strconv.ParseInt(string(b), 10, 64) + err := n.pulsar.Client.Get(endpoint, &result) + return result, err } func (n *namespaces) SetMaxConsumersPerSubscription(namespace utils.NameSpaceName, max int) error { @@ -500,12 +496,10 @@ func (n *namespaces) SetMaxConsumersPerSubscription(namespace utils.NameSpaceNam } func (n *namespaces) GetMaxConsumersPerSubscription(namespace utils.NameSpaceName) (int, error) { + var result int endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "maxConsumersPerSubscription") - b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) - if err != nil { - return -1, err - } - return strconv.Atoi(string(b)) + err := n.pulsar.Client.Get(endpoint, &result) + return result, err } func (n *namespaces) SetOffloadThreshold(namespace utils.NameSpaceName, threshold int64) error { @@ -514,12 +508,10 @@ func (n *namespaces) SetOffloadThreshold(namespace utils.NameSpaceName, threshol } func (n *namespaces) GetOffloadThreshold(namespace utils.NameSpaceName) (int64, error) { + var result int64 endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "offloadThreshold") - b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) - if err != nil { - return -1, err - } - return strconv.ParseInt(string(b), 10, 64) + err := n.pulsar.Client.Get(endpoint, &result) + return result, err } func (n *namespaces) SetMaxConsumersPerTopic(namespace utils.NameSpaceName, max int) error { @@ -528,12 +520,10 @@ func (n *namespaces) SetMaxConsumersPerTopic(namespace utils.NameSpaceName, max } func (n *namespaces) GetMaxConsumersPerTopic(namespace utils.NameSpaceName) (int, error) { + var result int endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "maxConsumersPerTopic") - b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) - if err != nil { - return -1, err - } - return strconv.Atoi(string(b)) + err := n.pulsar.Client.Get(endpoint, &result) + return result, err } func (n *namespaces) SetCompactionThreshold(namespace utils.NameSpaceName, threshold int64) error { @@ -542,12 +532,10 @@ func (n *namespaces) SetCompactionThreshold(namespace utils.NameSpaceName, thres } func (n *namespaces) GetCompactionThreshold(namespace utils.NameSpaceName) (int64, error) { + var result int64 endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "compactionThreshold") - b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) - if err != nil { - return -1, err - } - return strconv.ParseInt(string(b), 10, 64) + err := n.pulsar.Client.Get(endpoint, &result) + return result, err } func (n *namespaces) SetMaxProducersPerTopic(namespace utils.NameSpaceName, max int) error { @@ -556,12 +544,10 @@ func (n *namespaces) SetMaxProducersPerTopic(namespace utils.NameSpaceName, max } func (n *namespaces) GetMaxProducersPerTopic(namespace utils.NameSpaceName) (int, error) { + var result int endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "maxProducersPerTopic") - b, err := n.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) - if err != nil { - return -1, err - } - return strconv.Atoi(string(b)) + err := n.pulsar.Client.Get(endpoint, &result) + return result, err } func (n *namespaces) GetNamespaceReplicationClusters(namespace string) ([]string, error) { From c6c87901ca1ab748626e53459b0580471a2bb0da Mon Sep 17 00:00:00 2001 From: Jayson Reis Date: Thu, 4 Nov 2021 07:37:42 +0100 Subject: [PATCH 212/348] Remove omitempty from booleans (streamnative/pulsarctl#500) With omitempty it is impossible to set false on flags and they get removed from the final JSON Another change option would be making them pointers of boolean but it would break compatibility with current versions --- pulsaradmin/pkg/pulsar/utils/sink_config.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/utils/sink_config.go b/pulsaradmin/pkg/pulsar/utils/sink_config.go index 377fa5b629..033944d772 100644 --- a/pulsaradmin/pkg/pulsar/utils/sink_config.go +++ b/pulsaradmin/pkg/pulsar/utils/sink_config.go @@ -23,10 +23,10 @@ type SinkConfig struct { TimeoutMs *int64 `json:"timeoutMs,omitempty" yaml:"timeoutMs"` // Whether the subscriptions the functions created/used should be deleted when the functions is deleted - CleanupSubscription bool `json:"cleanupSubscription,omitempty" yaml:"cleanupSubscription"` + CleanupSubscription bool `json:"cleanupSubscription" yaml:"cleanupSubscription"` - RetainOrdering bool `json:"retainOrdering,omitempty" yaml:"retainOrdering"` - AutoAck bool `json:"autoAck,omitempty" yaml:"autoAck"` + RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` + AutoAck bool `json:"autoAck" yaml:"autoAck"` Parallelism int `json:"parallelism,omitempty" yaml:"parallelism"` Tenant string `json:"tenant,omitempty" yaml:"tenant"` Namespace string `json:"namespace,omitempty" yaml:"namespace"` From a8694b2fa142c14b3257cf49c5d4288e653461c6 Mon Sep 17 00:00:00 2001 From: Fushu Wang <93108425+FushuWang@users.noreply.github.com> Date: Thu, 25 Nov 2021 11:43:18 +0800 Subject: [PATCH 213/348] ParseAuthAction supports new actions (streamnative/pulsarctl#512) * chore: ignore vscode directory * feat: ParseAuthAction supports new actions * fix(permission): unit test case * docs(permission): update command help info of permission actions --- pulsaradmin/pkg/pulsar/common/auth_action.go | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/common/auth_action.go b/pulsaradmin/pkg/pulsar/common/auth_action.go index 2983c94f64..0a116a673e 100644 --- a/pulsaradmin/pkg/pulsar/common/auth_action.go +++ b/pulsaradmin/pkg/pulsar/common/auth_action.go @@ -25,6 +25,9 @@ const ( produce AuthAction = "produce" consume AuthAction = "consume" functionsAuth AuthAction = "functions" + packages AuthAction = "packages" + sinks AuthAction = "sinks" + sources AuthAction = "sources" ) func ParseAuthAction(action string) (AuthAction, error) { @@ -35,9 +38,15 @@ func ParseAuthAction(action string) (AuthAction, error) { return consume, nil case "functions": return functionsAuth, nil + case "packages": + return packages, nil + case "sinks": + return sinks, nil + case "sources": + return sources, nil default: return "", errors.Errorf("The auth action only can be specified as 'produce', "+ - "'consume', or 'functions'. Invalid auth action '%s'", action) + "'consume', 'sources', 'sinks', 'packages', or 'functions'. Invalid auth action '%s'", action) } } From 4cbb79af4f3ca810d023abd9ea2b6d296c1d3197 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 6 Dec 2021 15:02:05 +0800 Subject: [PATCH 214/348] fix: improve new client with auth provider (streamnative/pulsarctl#518) Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/auth/auth_provider.go | 24 +++++++++++++++++------- pulsaradmin/pkg/pulsar/admin.go | 21 +++++++++++++++++++-- 2 files changed, 36 insertions(+), 9 deletions(-) diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index c90e834c3d..89732c39dc 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -20,10 +20,8 @@ package auth import ( "crypto/tls" "crypto/x509" - "fmt" "io/ioutil" "net/http" - "os" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" ) @@ -41,8 +39,10 @@ type Transport struct { func GetAuthProvider(config *common.Config) (*Provider, error) { var provider Provider - defaultTransport := GetDefaultTransport(config) - var err error + defaultTransport, err := NewDefaultTransport(config) + if err != nil { + return nil, err + } switch config.AuthPlugin { case TLSPluginShortName: fallthrough @@ -68,7 +68,18 @@ func GetAuthProvider(config *common.Config) (*Provider, error) { return &provider, err } +// GetDefaultTransport gets a default transport. +// Deprecated: Use NewDefaultTransport instead. func GetDefaultTransport(config *common.Config) http.RoundTripper { + transport, err := NewDefaultTransport(config) + if err != nil { + panic(err) + } + + return transport +} + +func NewDefaultTransport(config *common.Config) (http.RoundTripper, error) { transport := http.DefaultTransport.(*http.Transport).Clone() tlsConfig := &tls.Config{ InsecureSkipVerify: config.TLSAllowInsecureConnection, @@ -76,13 +87,12 @@ func GetDefaultTransport(config *common.Config) http.RoundTripper { if len(config.TLSTrustCertsFilePath) > 0 { rootCA, err := ioutil.ReadFile(config.TLSTrustCertsFilePath) if err != nil { - fmt.Fprintln(os.Stderr, "error loading certificate authority:", err) - os.Exit(1) + return nil, err } tlsConfig.RootCAs = x509.NewCertPool() tlsConfig.RootCAs.AppendCertsFromPEM(rootCA) } transport.MaxIdleConnsPerHost = 10 transport.TLSClientConfig = tlsConfig - return transport + return transport, nil } diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index 5a2ffe807c..c72adee512 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -85,8 +85,24 @@ func New(config *common.Config) (Client, error) { return c, err } +// NewWithAuthProvider creates a client with auth provider. +// Deprecated: Use NewPulsarClientWithAuthProvider instead. func NewWithAuthProvider(config *common.Config, authProvider auth.Provider) Client { - defaultTransport := auth.GetDefaultTransport(config) + client, err := NewPulsarClientWithAuthProvider(config, authProvider) + if err != nil { + panic(err) + } + return client +} + +// NewPulsarClientWithAuthProvider create a client with auth provider. +func NewPulsarClientWithAuthProvider(config *common.Config, + authProvider auth.Provider) (Client, error) { + defaultTransport, err := auth.NewDefaultTransport(config) + if err != nil { + return nil, err + } + authProvider.WithTransport(defaultTransport) c := &pulsarClient{ @@ -100,7 +116,8 @@ func NewWithAuthProvider(config *common.Config, authProvider auth.Provider) Clie }, }, } - return c + + return c, nil } func (c *pulsarClient) endpoint(componentPath string, parts ...string) string { From fb8144341b3b3ec79c3a7580bd5e09845e55b1d6 Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Wed, 29 Dec 2021 10:31:42 +0800 Subject: [PATCH 215/348] bump pulsar-client-go/oauth2 (streamnative/pulsarctl#533) * update dependency to get rid of alert * update * tidy mod --- pulsaradmin/go.mod | 4 ++-- pulsaradmin/pkg/pulsar/token.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index 8d708bd2b7..e222a0d00c 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -4,10 +4,10 @@ go 1.13 require ( github.com/99designs/keyring v1.1.6 - github.com/apache/pulsar-client-go/oauth2 v0.0.0-20211006154457-742f1b107403 - github.com/dgrijalva/jwt-go v3.2.0+incompatible + github.com/apache/pulsar-client-go/oauth2 v0.0.0-20211108044248-fe3b7c4e445b github.com/docker/go-connections v0.4.0 github.com/fatih/color v1.7.0 + github.com/form3tech-oss/jwt-go v3.2.3+incompatible github.com/ghodss/yaml v1.0.0 github.com/go-sql-driver/mysql v1.5.0 // indirect github.com/gogo/protobuf v1.3.2 // indirect diff --git a/pulsaradmin/pkg/pulsar/token.go b/pulsaradmin/pkg/pulsar/token.go index 0ac946185d..1c2db44495 100644 --- a/pulsaradmin/pkg/pulsar/token.go +++ b/pulsaradmin/pkg/pulsar/token.go @@ -23,7 +23,7 @@ import ( "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/algorithm" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" - "github.com/dgrijalva/jwt-go" + "github.com/form3tech-oss/jwt-go" "github.com/pkg/errors" ) From 39777320e51d47209b0689c2ed483adf71134dba Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 11 Jan 2022 15:33:57 +0800 Subject: [PATCH 216/348] fix: update type in namespace policies (streamnative/pulsarctl#556) Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/pulsar/utils/policies.go | 26 ++++++++++++------------ 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/utils/policies.go b/pulsaradmin/pkg/pulsar/utils/policies.go index 52de13a935..9b59eb5b5e 100644 --- a/pulsaradmin/pkg/pulsar/utils/policies.go +++ b/pulsaradmin/pkg/pulsar/utils/policies.go @@ -31,16 +31,16 @@ type Policies struct { Persistence *PersistencePolicies `json:"persistence"` RetentionPolicies *RetentionPolicies `json:"retention_policies"` SchemaValidationEnforced bool `json:"schema_validation_enforced"` - DeduplicationEnabled bool `json:"deduplicationEnabled"` + DeduplicationEnabled *bool `json:"deduplicationEnabled"` Deleted bool `json:"deleted"` EncryptionRequired bool `json:"encryption_required"` - MessageTTLInSeconds int `json:"message_ttl_in_seconds"` - MaxProducersPerTopic int `json:"max_producers_per_topic"` - MaxConsumersPerTopic int `json:"max_consumers_per_topic"` - MaxConsumersPerSubscription int `json:"max_consumers_per_subscription"` - CompactionThreshold int64 `json:"compaction_threshold"` + MessageTTLInSeconds *int `json:"message_ttl_in_seconds"` + MaxProducersPerTopic *int `json:"max_producers_per_topic"` + MaxConsumersPerTopic *int `json:"max_consumers_per_topic"` + MaxConsumersPerSubscription *int `json:"max_consumers_per_subscription"` + CompactionThreshold *int64 `json:"compaction_threshold"` OffloadThreshold int64 `json:"offload_threshold"` - OffloadDeletionLagMs int64 `json:"offload_deletion_lag_ms"` + OffloadDeletionLagMs *int64 `json:"offload_deletion_lag_ms"` AntiAffinityGroup string `json:"antiAffinityGroup"` ReplicationClusters []string `json:"replication_clusters"` LatencyStatsSampleRate map[string]int `json:"latency_stats_sample_rate"` @@ -50,7 +50,7 @@ type Policies struct { ReplicatorDispatchRate map[string]DispatchRate `json:"replicatorDispatchRate"` PublishMaxMessageRate map[string]PublishRate `json:"publishMaxMessageRate"` ClusterSubscribeRate map[string]SubscribeRate `json:"clusterSubscribeRate"` - TopicAutoCreationConfig TopicAutoCreationConfig `json:"autoTopicCreationOverride"` + TopicAutoCreationConfig *TopicAutoCreationConfig `json:"autoTopicCreationOverride"` SchemaCompatibilityStrategy SchemaCompatibilityStrategy `json:"schema_auto_update_compatibility_strategy"` AuthPolicies common.AuthPolicies `json:"auth_policies"` SubscriptionAuthMode SubscriptionAuthMode `json:"subscription_auth_mode"` @@ -67,14 +67,14 @@ func NewDefaultPolicies() *Policies { PublishMaxMessageRate: make(map[string]PublishRate), ClusterSubscribeRate: make(map[string]SubscribeRate), LatencyStatsSampleRate: make(map[string]int), - MessageTTLInSeconds: 0, + MessageTTLInSeconds: nil, Deleted: false, EncryptionRequired: false, SubscriptionAuthMode: None, - MaxProducersPerTopic: 0, - MaxConsumersPerSubscription: 0, - MaxConsumersPerTopic: 0, - CompactionThreshold: 0, + MaxProducersPerTopic: nil, + MaxConsumersPerSubscription: nil, + MaxConsumersPerTopic: nil, + CompactionThreshold: nil, OffloadThreshold: -1, SchemaCompatibilityStrategy: Full, SchemaValidationEnforced: false, From d30fc3723ddac9ecaa8d92cf85276a3db20067bb Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 24 Jan 2022 20:06:52 +0800 Subject: [PATCH 217/348] feat: add allow auto update schema cmd (streamnative/pulsarctl#567) Signed-off-by: Zixuan Liu --- pulsaradmin/go.mod | 8 +++++--- pulsaradmin/pkg/pulsar/namespace.go | 18 ++++++++++++++++++ pulsaradmin/pkg/pulsar/utils/policies.go | 1 + 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index e222a0d00c..5d9207ceca 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -11,7 +11,7 @@ require ( github.com/ghodss/yaml v1.0.0 github.com/go-sql-driver/mysql v1.5.0 // indirect github.com/gogo/protobuf v1.3.2 // indirect - github.com/golang/protobuf v1.4.3 + github.com/golang/protobuf v1.5.2 github.com/gorilla/mux v1.7.4 // indirect github.com/imdario/mergo v0.3.8 github.com/kr/pretty v0.2.0 // indirect @@ -21,17 +21,19 @@ require ( github.com/mattn/go-colorable v0.1.2 // indirect github.com/mattn/go-runewidth v0.0.4 // indirect github.com/olekukonko/tablewriter v0.0.1 + github.com/onsi/gomega v1.18.0 github.com/pkg/errors v0.9.1 github.com/sirupsen/logrus v1.4.2 // indirect github.com/spf13/cobra v0.0.5 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.5.1 github.com/testcontainers/testcontainers-go v0.0.10 - golang.org/x/net v0.0.0-20210220033124-5f55cee0dc0d // indirect golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93 google.golang.org/appengine v1.6.7 // indirect gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 // indirect - gopkg.in/yaml.v2 v2.3.0 + gopkg.in/yaml.v2 v2.4.0 ) replace github.com/apache/pulsar-client-go/oauth2 => github.com/apache/pulsar-client-go/oauth2 v0.0.0-20211006154457-742f1b107403 + +replace golang.org/x/sys => golang.org/x/sys v0.0.0-20201119102817-f84b799fce68 diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index efeee0558f..a5dd5a414d 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -259,6 +259,12 @@ type Namespaces interface { // GetPublishRate gets the maximum rate or number of messages that producer can publish to topics in the namespace GetPublishRate(namespace utils.NameSpaceName) (utils.PublishRate, error) + + // SetIsAllowAutoUpdateSchema sets whether to allow auto update schema on a namespace + SetIsAllowAutoUpdateSchema(namespace utils.NameSpaceName, isAllowAutoUpdateSchema bool) error + + // GetIsAllowAutoUpdateSchema gets whether to allow auto update schema on a namespace + GetIsAllowAutoUpdateSchema(namespace utils.NameSpaceName) (bool, error) } type namespaces struct { @@ -830,3 +836,15 @@ func (n *namespaces) GetPublishRate(namespace utils.NameSpaceName) (utils.Publis err := n.pulsar.Client.Get(endpoint, &pubRate) return pubRate, err } + +func (n *namespaces) SetIsAllowAutoUpdateSchema(namespace utils.NameSpaceName, isAllowAutoUpdateSchema bool) error { + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "isAllowAutoUpdateSchema") + return n.pulsar.Client.Post(endpoint, &isAllowAutoUpdateSchema) +} + +func (n *namespaces) GetIsAllowAutoUpdateSchema(namespace utils.NameSpaceName) (bool, error) { + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "isAllowAutoUpdateSchema") + var result bool + err := n.pulsar.Client.Get(endpoint, &result) + return result, err +} diff --git a/pulsaradmin/pkg/pulsar/utils/policies.go b/pulsaradmin/pkg/pulsar/utils/policies.go index 9b59eb5b5e..9822bb0c65 100644 --- a/pulsaradmin/pkg/pulsar/utils/policies.go +++ b/pulsaradmin/pkg/pulsar/utils/policies.go @@ -54,6 +54,7 @@ type Policies struct { SchemaCompatibilityStrategy SchemaCompatibilityStrategy `json:"schema_auto_update_compatibility_strategy"` AuthPolicies common.AuthPolicies `json:"auth_policies"` SubscriptionAuthMode SubscriptionAuthMode `json:"subscription_auth_mode"` + IsAllowAutoUpdateSchema *bool `json:"is_allow_auto_update_schema"` } func NewDefaultPolicies() *Policies { From f32046d9c93d66441715feda7ab873df1f4bdd86 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 11 Feb 2022 13:56:37 +0800 Subject: [PATCH 218/348] fix: create non-partitions topic (streamnative/pulsarctl#577) Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/pulsar/topic.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index f949d1a0cb..814e16e2f7 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -250,10 +250,13 @@ func (c *pulsarClient) Topics() Topics { func (t *topics) Create(topic utils.TopicName, partitions int) error { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "partitions") + data := &partitions if partitions == 0 { endpoint = t.pulsar.endpoint(t.basePath, topic.GetRestPath()) + data = nil } - return t.pulsar.Client.Put(endpoint, partitions) + + return t.pulsar.Client.Put(endpoint, data) } func (t *topics) Delete(topic utils.TopicName, force bool, nonPartitioned bool) error { From 725f1bda55ef4f7119c00a48fea192e33e0cabad Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 18 Feb 2022 10:59:51 +0800 Subject: [PATCH 219/348] fix: fix encode json body (streamnative/pulsarctl#610) * fix: fix encode json body Signed-off-by: Zixuan Liu * test: fix get_ns_anti_affinity_group test Signed-off-by: Zixuan Liu * chore: fix license Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/cli/client.go | 7 ++--- pulsaradmin/pkg/cli/client_test.go | 47 ++++++++++++++++++++++++++++++ 2 files changed, 50 insertions(+), 4 deletions(-) create mode 100644 pulsaradmin/pkg/cli/client_test.go diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/cli/client.go index e3775f5ed8..42f9147667 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/cli/client.go @@ -364,12 +364,11 @@ func endpoint(parts ...string) string { // encodeJSONBody is used to JSON encode a body func encodeJSONBody(obj interface{}) (io.Reader, error) { - buf := bytes.NewBuffer(nil) - enc := json.NewEncoder(buf) - if err := enc.Encode(obj); err != nil { + b, err := json.Marshal(obj) + if err != nil { return nil, err } - return buf, nil + return bytes.NewReader(b), nil } // decodeJSONBody is used to JSON decode a body diff --git a/pulsaradmin/pkg/cli/client_test.go b/pulsaradmin/pkg/cli/client_test.go new file mode 100644 index 0000000000..f5335946b7 --- /dev/null +++ b/pulsaradmin/pkg/cli/client_test.go @@ -0,0 +1,47 @@ +// 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 cli + +import ( + "io/ioutil" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestEncodeJSONBody(t *testing.T) { + testcases := []struct { + obj interface{} + expected int + }{ + {obj: "1", expected: 3}, + {obj: "12", expected: 4}, + {obj: 1, expected: 1}, + {obj: 12, expected: 2}, + } + + for _, testcase := range testcases { + r, err := encodeJSONBody(testcase.obj) + require.NoError(t, err) + + b, err := ioutil.ReadAll(r) + require.NoError(t, err) + + require.Equal(t, testcase.expected, len(b)) + } +} From 3a9f0e28310ed8254790c12608c19dca45702bfe Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 29 Apr 2022 23:18:07 +0800 Subject: [PATCH 220/348] fix: use nil instead of empty string as request body (streamnative/pulsarctl#662) Signed-off-by: Zixuan Liu Co-authored-by: lipenghui --- pulsaradmin/pkg/pulsar/functions.go | 12 ++++++------ pulsaradmin/pkg/pulsar/namespace.go | 18 +++++++++--------- pulsaradmin/pkg/pulsar/sinks.go | 14 +++++++------- pulsaradmin/pkg/pulsar/sources.go | 14 +++++++------- pulsaradmin/pkg/pulsar/subscription.go | 10 +++++----- pulsaradmin/pkg/pulsar/topic.go | 6 +++--- 6 files changed, 37 insertions(+), 37 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/pulsar/functions.go index 77807fd8b7..9af8216e7d 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/pulsar/functions.go @@ -266,14 +266,14 @@ func (f *functions) CreateFuncWithURL(funcConf *utils.FunctionConfig, pkgURL str func (f *functions) StopFunction(tenant, namespace, name string) error { endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name) - return f.pulsar.Client.Post(endpoint+"/stop", "") + return f.pulsar.Client.Post(endpoint+"/stop", nil) } func (f *functions) StopFunctionWithID(tenant, namespace, name string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name, id) - return f.pulsar.Client.Post(endpoint+"/stop", "") + return f.pulsar.Client.Post(endpoint+"/stop", nil) } func (f *functions) DeleteFunction(tenant, namespace, name string) error { @@ -329,26 +329,26 @@ func (f *functions) DownloadFunctionByNs(destinationFile, tenant, namespace, fun func (f *functions) StartFunction(tenant, namespace, name string) error { endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name) - return f.pulsar.Client.Post(endpoint+"/start", "") + return f.pulsar.Client.Post(endpoint+"/start", nil) } func (f *functions) StartFunctionWithID(tenant, namespace, name string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name, id) - return f.pulsar.Client.Post(endpoint+"/start", "") + return f.pulsar.Client.Post(endpoint+"/start", nil) } func (f *functions) RestartFunction(tenant, namespace, name string) error { endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name) - return f.pulsar.Client.Post(endpoint+"/restart", "") + return f.pulsar.Client.Post(endpoint+"/restart", nil) } func (f *functions) RestartFunctionWithID(tenant, namespace, name string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := f.pulsar.endpoint(f.basePath, tenant, namespace, name, id) - return f.pulsar.Client.Post(endpoint+"/restart", "") + return f.pulsar.Client.Post(endpoint+"/restart", nil) } func (f *functions) GetFunctions(tenant, namespace string) ([]string, error) { diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/pulsar/namespace.go index a5dd5a414d..771ef6ac03 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/pulsar/namespace.go @@ -678,7 +678,7 @@ func (n *namespaces) Unload(namespace string) error { return err } endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), "unload") - return n.pulsar.Client.Put(endpoint, "") + return n.pulsar.Client.Put(endpoint, nil) } func (n *namespaces) UnloadNamespaceBundle(namespace, bundle string) error { @@ -687,7 +687,7 @@ func (n *namespaces) UnloadNamespaceBundle(namespace, bundle string) error { return err } endpoint := n.pulsar.endpoint(n.basePath, nsName.String(), bundle, "unload") - return n.pulsar.Client.Put(endpoint, "") + return n.pulsar.Client.Put(endpoint, nil) } func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitBundles bool) error { @@ -699,7 +699,7 @@ func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitB params := map[string]string{ "unload": strconv.FormatBool(unloadSplitBundles), } - return n.pulsar.Client.PutWithQueryParams(endpoint, "", nil, params) + return n.pulsar.Client.PutWithQueryParams(endpoint, nil, nil, params) } func (n *namespaces) GetNamespacePermissions(namespace utils.NameSpaceName) (map[string][]common.AuthAction, error) { @@ -748,33 +748,33 @@ func (n *namespaces) SetEncryptionRequiredStatus(namespace utils.NameSpaceName, func (n *namespaces) UnsubscribeNamespace(namespace utils.NameSpaceName, sName string) error { endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "unsubscribe", url.QueryEscape(sName)) - return n.pulsar.Client.Post(endpoint, "") + return n.pulsar.Client.Post(endpoint, nil) } func (n *namespaces) UnsubscribeNamespaceBundle(namespace utils.NameSpaceName, bundle, sName string) error { endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), bundle, "unsubscribe", url.QueryEscape(sName)) - return n.pulsar.Client.Post(endpoint, "") + return n.pulsar.Client.Post(endpoint, nil) } func (n *namespaces) ClearNamespaceBundleBacklogForSubscription(namespace utils.NameSpaceName, bundle, sName string) error { endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog", url.QueryEscape(sName)) - return n.pulsar.Client.Post(endpoint, "") + return n.pulsar.Client.Post(endpoint, nil) } func (n *namespaces) ClearNamespaceBundleBacklog(namespace utils.NameSpaceName, bundle string) error { endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), bundle, "clearBacklog") - return n.pulsar.Client.Post(endpoint, "") + return n.pulsar.Client.Post(endpoint, nil) } func (n *namespaces) ClearNamespaceBacklogForSubscription(namespace utils.NameSpaceName, sName string) error { endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "clearBacklog", url.QueryEscape(sName)) - return n.pulsar.Client.Post(endpoint, "") + return n.pulsar.Client.Post(endpoint, nil) } func (n *namespaces) ClearNamespaceBacklog(namespace utils.NameSpaceName) error { endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "clearBacklog") - return n.pulsar.Client.Post(endpoint, "") + return n.pulsar.Client.Post(endpoint, nil) } func (n *namespaces) SetReplicatorDispatchRate(namespace utils.NameSpaceName, rate utils.DispatchRate) error { diff --git a/pulsaradmin/pkg/pulsar/sinks.go b/pulsaradmin/pkg/pulsar/sinks.go index 22eb819baf..194b1d6ff2 100644 --- a/pulsaradmin/pkg/pulsar/sinks.go +++ b/pulsaradmin/pkg/pulsar/sinks.go @@ -390,38 +390,38 @@ func (s *sinks) GetSinkStatusWithID(tenant, namespace, sink string, id int) (uti func (s *sinks) RestartSink(tenant, namespace, sink string) error { endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink) - return s.pulsar.Client.Post(endpoint+"/restart", "") + return s.pulsar.Client.Post(endpoint+"/restart", nil) } func (s *sinks) RestartSinkWithID(tenant, namespace, sink string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink, id) - return s.pulsar.Client.Post(endpoint+"/restart", "") + return s.pulsar.Client.Post(endpoint+"/restart", nil) } func (s *sinks) StopSink(tenant, namespace, sink string) error { endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink) - return s.pulsar.Client.Post(endpoint+"/stop", "") + return s.pulsar.Client.Post(endpoint+"/stop", nil) } func (s *sinks) StopSinkWithID(tenant, namespace, sink string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink, id) - return s.pulsar.Client.Post(endpoint+"/stop", "") + return s.pulsar.Client.Post(endpoint+"/stop", nil) } func (s *sinks) StartSink(tenant, namespace, sink string) error { endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink) - return s.pulsar.Client.Post(endpoint+"/start", "") + return s.pulsar.Client.Post(endpoint+"/start", nil) } func (s *sinks) StartSinkWithID(tenant, namespace, sink string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, sink, id) - return s.pulsar.Client.Post(endpoint+"/start", "") + return s.pulsar.Client.Post(endpoint+"/start", nil) } func (s *sinks) GetBuiltInSinks() ([]*utils.ConnectorDefinition, error) { @@ -433,5 +433,5 @@ func (s *sinks) GetBuiltInSinks() ([]*utils.ConnectorDefinition, error) { func (s *sinks) ReloadBuiltInSinks() error { endpoint := s.pulsar.endpoint(s.basePath, "reloadBuiltInSinks") - return s.pulsar.Client.Post(endpoint, "") + return s.pulsar.Client.Post(endpoint, nil) } diff --git a/pulsaradmin/pkg/pulsar/sources.go b/pulsaradmin/pkg/pulsar/sources.go index fd2d29a854..1899e6e8d2 100644 --- a/pulsaradmin/pkg/pulsar/sources.go +++ b/pulsaradmin/pkg/pulsar/sources.go @@ -393,38 +393,38 @@ func (s *sources) GetSourceStatusWithID(tenant, namespace, source string, id int func (s *sources) RestartSource(tenant, namespace, source string) error { endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source) - return s.pulsar.Client.Post(endpoint+"/restart", "") + return s.pulsar.Client.Post(endpoint+"/restart", nil) } func (s *sources) RestartSourceWithID(tenant, namespace, source string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source, id) - return s.pulsar.Client.Post(endpoint+"/restart", "") + return s.pulsar.Client.Post(endpoint+"/restart", nil) } func (s *sources) StopSource(tenant, namespace, source string) error { endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source) - return s.pulsar.Client.Post(endpoint+"/stop", "") + return s.pulsar.Client.Post(endpoint+"/stop", nil) } func (s *sources) StopSourceWithID(tenant, namespace, source string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source, id) - return s.pulsar.Client.Post(endpoint+"/stop", "") + return s.pulsar.Client.Post(endpoint+"/stop", nil) } func (s *sources) StartSource(tenant, namespace, source string) error { endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source) - return s.pulsar.Client.Post(endpoint+"/start", "") + return s.pulsar.Client.Post(endpoint+"/start", nil) } func (s *sources) StartSourceWithID(tenant, namespace, source string, instanceID int) error { id := fmt.Sprintf("%d", instanceID) endpoint := s.pulsar.endpoint(s.basePath, tenant, namespace, source, id) - return s.pulsar.Client.Post(endpoint+"/start", "") + return s.pulsar.Client.Post(endpoint+"/start", nil) } func (s *sources) GetBuiltInSources() ([]*utils.ConnectorDefinition, error) { @@ -436,5 +436,5 @@ func (s *sources) GetBuiltInSources() ([]*utils.ConnectorDefinition, error) { func (s *sources) ReloadBuiltInSources() error { endpoint := s.pulsar.endpoint(s.basePath, "reloadBuiltInSources") - return s.pulsar.Client.Post(endpoint, "") + return s.pulsar.Client.Post(endpoint, nil) } diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/pulsar/subscription.go index bffffc181e..2aeb2297e2 100644 --- a/pulsaradmin/pkg/pulsar/subscription.go +++ b/pulsaradmin/pkg/pulsar/subscription.go @@ -111,34 +111,34 @@ func (s *subscriptions) ResetCursorToTimestamp(topic utils.TopicName, sName stri endpoint := s.pulsar.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName), "resetcursor", strconv.FormatInt(timestamp, 10)) - return s.pulsar.Client.Post(endpoint, "") + return s.pulsar.Client.Post(endpoint, nil) } func (s *subscriptions) ClearBacklog(topic utils.TopicName, sName string) error { endpoint := s.pulsar.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName), "skip_all") - return s.pulsar.Client.Post(endpoint, "") + return s.pulsar.Client.Post(endpoint, nil) } func (s *subscriptions) SkipMessages(topic utils.TopicName, sName string, n int64) error { endpoint := s.pulsar.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName), "skip", strconv.FormatInt(n, 10)) - return s.pulsar.Client.Post(endpoint, "") + return s.pulsar.Client.Post(endpoint, nil) } func (s *subscriptions) ExpireMessages(topic utils.TopicName, sName string, expire int64) error { endpoint := s.pulsar.endpoint( s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName), "expireMessages", strconv.FormatInt(expire, 10)) - return s.pulsar.Client.Post(endpoint, "") + return s.pulsar.Client.Post(endpoint, nil) } func (s *subscriptions) ExpireAllMessages(topic utils.TopicName, expire int64) error { endpoint := s.pulsar.endpoint( s.basePath, topic.GetRestPath(), "all_subscription", "expireMessages", strconv.FormatInt(expire, 10)) - return s.pulsar.Client.Post(endpoint, "") + return s.pulsar.Client.Post(endpoint, nil) } func (s *subscriptions) PeekMessages(topic utils.TopicName, sName string, n int) ([]*utils.Message, error) { diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index 814e16e2f7..b0f0058bcc 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -401,7 +401,7 @@ func (t *topics) GetPartitionedStats(topic utils.TopicName, perPartition bool) ( func (t *topics) Terminate(topic utils.TopicName) (utils.MessageID, error) { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "terminate") var messageID utils.MessageID - err := t.pulsar.Client.PostWithObj(endpoint, "", &messageID) + err := t.pulsar.Client.PostWithObj(endpoint, nil, &messageID) return messageID, err } @@ -419,12 +419,12 @@ func (t *topics) OffloadStatus(topic utils.TopicName) (utils.OffloadProcessStatu func (t *topics) Unload(topic utils.TopicName) error { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "unload") - return t.pulsar.Client.Put(endpoint, "") + return t.pulsar.Client.Put(endpoint, nil) } func (t *topics) Compact(topic utils.TopicName) error { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "compaction") - return t.pulsar.Client.Put(endpoint, "") + return t.pulsar.Client.Put(endpoint, nil) } func (t *topics) CompactStatus(topic utils.TopicName) (utils.LongRunningProcessStatus, error) { From ca8e2d88a02da6516413628e9ff0fb5a53481da1 Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Thu, 5 May 2022 23:07:02 +0800 Subject: [PATCH 221/348] support build with go 1.18 (streamnative/pulsarctl#699) * support build with go 1.18 * update to 1.18 * drop darwin/386 support via go 1.15 * change 1.13 to 1.18 * use latest version of golangci-lint * fix style * address comment * address comment * address comment * fix --- pulsaradmin/go.mod | 54 +++++++++++++++---- pulsaradmin/pkg/pulsar/subscription.go | 2 +- pulsaradmin/pkg/pulsar/tenant.go | 2 +- .../pkg/pulsar/utils/namespace_name.go | 7 +-- pulsaradmin/pkg/pulsar/utils/resources.go | 2 +- .../pkg/pulsar/utils/topics_stats_stream.go | 2 +- 6 files changed, 49 insertions(+), 20 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index 5d9207ceca..a187606338 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -1,6 +1,6 @@ module github.com/streamnative/pulsar-admin-go -go 1.13 +go 1.18 require ( github.com/99designs/keyring v1.1.6 @@ -9,31 +9,65 @@ require ( github.com/fatih/color v1.7.0 github.com/form3tech-oss/jwt-go v3.2.3+incompatible github.com/ghodss/yaml v1.0.0 - github.com/go-sql-driver/mysql v1.5.0 // indirect - github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.5.2 - github.com/gorilla/mux v1.7.4 // indirect github.com/imdario/mergo v0.3.8 - github.com/kr/pretty v0.2.0 // indirect github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b github.com/magiconair/properties v1.8.0 - github.com/mattn/go-colorable v0.1.2 // indirect - github.com/mattn/go-runewidth v0.0.4 // indirect github.com/olekukonko/tablewriter v0.0.1 github.com/onsi/gomega v1.18.0 github.com/pkg/errors v0.9.1 - github.com/sirupsen/logrus v1.4.2 // indirect github.com/spf13/cobra v0.0.5 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.5.1 github.com/testcontainers/testcontainers-go v0.0.10 golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93 + gopkg.in/yaml.v2 v2.4.0 +) + +require ( + github.com/Microsoft/go-winio v0.4.11 // indirect + github.com/Microsoft/hcsshim v0.8.6 // indirect + github.com/cenkalti/backoff v2.2.1+incompatible // indirect + github.com/containerd/continuity v0.0.0-20190426062206-aaeac12a7ffc // indirect + github.com/danieljoos/wincred v1.0.2 // indirect + github.com/davecgh/go-spew v1.1.1 // indirect + github.com/docker/distribution v2.7.1-0.20190205005809-0d3efadf0154+incompatible // indirect + github.com/docker/docker v0.7.3-0.20190506211059-b20a14b54661 // indirect + github.com/docker/go-units v0.3.3 // indirect + github.com/dvsekhvalnov/jose2go v0.0.0-20200901110807-248326c1351b // indirect + github.com/go-sql-driver/mysql v1.5.0 // indirect + github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/gorilla/mux v1.7.4 // indirect + github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect + github.com/inconshreveable/mousetrap v1.0.0 // indirect + github.com/keybase/go-keychain v0.0.0-20190712205309-48d3d31d256d // indirect + github.com/konsorten/go-windows-terminal-sequences v1.0.1 // indirect + github.com/kr/pretty v0.2.0 // indirect + github.com/mattn/go-colorable v0.1.2 // indirect + github.com/mattn/go-isatty v0.0.8 // indirect + github.com/mattn/go-runewidth v0.0.4 // indirect + github.com/mitchellh/go-homedir v1.1.0 // indirect + github.com/mtibben/percent v0.2.1 // indirect + github.com/opencontainers/go-digest v1.0.0-rc1 // indirect + github.com/opencontainers/image-spec v1.0.1 // indirect + github.com/opencontainers/runc v0.1.1 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/satori/go.uuid v1.2.0 // indirect + github.com/sirupsen/logrus v1.4.2 // indirect + golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 // indirect + golang.org/x/net v0.0.0-20210428140749-89ef3d95e781 // indirect + golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9 // indirect + golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 // indirect + golang.org/x/text v0.3.6 // indirect google.golang.org/appengine v1.6.7 // indirect + google.golang.org/genproto v0.0.0-20200825200019-8632dd797987 // indirect + google.golang.org/grpc v1.31.0 // indirect + google.golang.org/protobuf v1.26.0 // indirect gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 // indirect - gopkg.in/yaml.v2 v2.4.0 ) replace github.com/apache/pulsar-client-go/oauth2 => github.com/apache/pulsar-client-go/oauth2 v0.0.0-20211006154457-742f1b107403 -replace golang.org/x/sys => golang.org/x/sys v0.0.0-20201119102817-f84b799fce68 +replace golang.org/x/sys => golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/pulsar/subscription.go index 2aeb2297e2..42d1be8c39 100644 --- a/pulsaradmin/pkg/pulsar/subscription.go +++ b/pulsaradmin/pkg/pulsar/subscription.go @@ -261,7 +261,7 @@ func getIndividualMsgsFromBatch(topic utils.TopicName, msgID *utils.MessageID, d } } - //payload + // payload singlePayload := make([]byte, singleMeta.GetPayloadSize()) if _, err := io.ReadFull(rdBuf, singlePayload); err != nil { return nil, err diff --git a/pulsaradmin/pkg/pulsar/tenant.go b/pulsaradmin/pkg/pulsar/tenant.go index 84e133c8cf..2d64540ff3 100644 --- a/pulsaradmin/pkg/pulsar/tenant.go +++ b/pulsaradmin/pkg/pulsar/tenant.go @@ -32,7 +32,7 @@ type Tenants interface { // Update the admins for a tenant Update(utils.TenantData) error - //List returns the list of tenants + // List returns the list of tenants List() ([]string, error) // Get returns the config of the tenant. diff --git a/pulsaradmin/pkg/pulsar/utils/namespace_name.go b/pulsaradmin/pkg/pulsar/utils/namespace_name.go index f35c66eef0..59fb8d4796 100644 --- a/pulsaradmin/pkg/pulsar/utils/namespace_name.go +++ b/pulsaradmin/pkg/pulsar/utils/namespace_name.go @@ -81,13 +81,8 @@ func validateNamespaceName(tenant, namespace string) error { // allowed characters for property, namespace, cluster and topic // names are alphanumeric (a-zA-Z0-9) and these special chars -=:. // and % is allowed as part of valid URL encoding -const PATTEN = "^[-=:.\\w]*$" +var patten = regexp.MustCompile("^[-=:.\\w]*$") func CheckName(name string) bool { - patten, err := regexp.Compile(PATTEN) - if err != nil { - return false - } - return patten.MatchString(name) } diff --git a/pulsaradmin/pkg/pulsar/utils/resources.go b/pulsaradmin/pkg/pulsar/utils/resources.go index f5da7b87a1..d5b7f3f074 100644 --- a/pulsaradmin/pkg/pulsar/utils/resources.go +++ b/pulsaradmin/pkg/pulsar/utils/resources.go @@ -25,7 +25,7 @@ type Resources struct { func NewDefaultResources() *Resources { resources := &Resources{ - //Default cpu is 1 core + // Default cpu is 1 core CPU: 1, // Default memory is 1GB Disk: 1073741824, diff --git a/pulsaradmin/pkg/pulsar/utils/topics_stats_stream.go b/pulsaradmin/pkg/pulsar/utils/topics_stats_stream.go index 1af615a2d2..7554609e6e 100644 --- a/pulsaradmin/pkg/pulsar/utils/topics_stats_stream.go +++ b/pulsaradmin/pkg/pulsar/utils/topics_stats_stream.go @@ -17,7 +17,7 @@ package utils -//var TopicsMap map[string]map[string]map[string]TopicStats +// var TopicsMap map[string]map[string]map[string]TopicStats type TopicStatsStream struct { TopicsMap map[string]map[string]map[string]TopicStats `json:"topicStatsBuf"` From 342f1d67026880410b02fde6d3757992ea6c3e74 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 6 Jun 2022 18:04:37 +0800 Subject: [PATCH 222/348] fix: fix tls insecure connection (streamnative/pulsarctl#731) Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/auth/auth_provider.go | 4 +- pulsaradmin/pkg/pulsar/admin.go | 53 +++++++-------- pulsaradmin/pkg/pulsar/admin_test.go | 75 +++++++++++++++++++++- pulsaradmin/pkg/pulsar/utils/utils_test.go | 6 ++ 4 files changed, 109 insertions(+), 29 deletions(-) diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index 89732c39dc..17bb69e06b 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -37,7 +37,7 @@ type Transport struct { T http.RoundTripper } -func GetAuthProvider(config *common.Config) (*Provider, error) { +func GetAuthProvider(config *common.Config) (Provider, error) { var provider Provider defaultTransport, err := NewDefaultTransport(config) if err != nil { @@ -65,7 +65,7 @@ func GetAuthProvider(config *common.Config) (*Provider, error) { config.IssuerEndpoint, config.ClientID, config.Audience, config.Scope, defaultTransport) } } - return &provider, err + return provider, err } // GetDefaultTransport gets a default transport. diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/pulsar/admin.go index c72adee512..5f3eee3419 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/pulsar/admin.go @@ -18,7 +18,6 @@ package pulsar import ( - "fmt" "net/http" "net/url" "path" @@ -61,28 +60,11 @@ type pulsarClient struct { // New returns a new client func New(config *common.Config) (Client, error) { - if len(config.WebServiceURL) == 0 { - config.WebServiceURL = DefaultWebServiceURL - } - - c := &pulsarClient{ - APIVersion: config.PulsarAPIVersion, - Client: &cli.Client{ - ServiceURL: config.WebServiceURL, - VersionInfo: ReleaseVersion, - HTTPClient: &http.Client{ - Timeout: DefaultHTTPTimeOutDuration, - }, - }, - } - authProvider, err := auth.GetAuthProvider(config) - if !utils.IsNilFixed(authProvider) { - c.Client.HTTPClient.Transport = *authProvider - } else { - fmt.Printf("No Auth Provider found\n") + if err != nil { + return nil, err } - return c, err + return NewPulsarClientWithAuthProvider(config, authProvider) } // NewWithAuthProvider creates a client with auth provider. @@ -98,12 +80,31 @@ func NewWithAuthProvider(config *common.Config, authProvider auth.Provider) Clie // NewPulsarClientWithAuthProvider create a client with auth provider. func NewPulsarClientWithAuthProvider(config *common.Config, authProvider auth.Provider) (Client, error) { - defaultTransport, err := auth.NewDefaultTransport(config) - if err != nil { - return nil, err + var transport http.RoundTripper + + if authProvider != nil { + transport = authProvider.Transport() + if transport != nil { + transport = authProvider + } + } + + if transport == nil { + defaultTransport, err := auth.NewDefaultTransport(config) + if err != nil { + return nil, err + } + if authProvider != nil { + authProvider.WithTransport(authProvider) + } else { + transport = defaultTransport + } } - authProvider.WithTransport(defaultTransport) + webServiceURL := config.WebServiceURL + if len(webServiceURL) == 0 { + config.WebServiceURL = DefaultWebServiceURL + } c := &pulsarClient{ APIVersion: config.PulsarAPIVersion, @@ -112,7 +113,7 @@ func NewPulsarClientWithAuthProvider(config *common.Config, VersionInfo: ReleaseVersion, HTTPClient: &http.Client{ Timeout: DefaultHTTPTimeOutDuration, - Transport: authProvider, + Transport: transport, }, }, } diff --git a/pulsaradmin/pkg/pulsar/admin_test.go b/pulsaradmin/pkg/pulsar/admin_test.go index 91268d9846..194b4b520e 100644 --- a/pulsaradmin/pkg/pulsar/admin_test.go +++ b/pulsaradmin/pkg/pulsar/admin_test.go @@ -18,11 +18,13 @@ package pulsar import ( + "net/http" "testing" + "github.com/streamnative/pulsar-admin-go/pkg/auth" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" - "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestPulsarClientEndpointEscapes(t *testing.T) { @@ -31,3 +33,74 @@ func TestPulsarClientEndpointEscapes(t *testing.T) { expected := "/admin/v2/myendpoint/abc%25%3F%20%2Fdef/ghi" assert.Equal(t, expected, actual) } + +func TestNew(t *testing.T) { + config := &common.Config{} + admin, err := New(config) + require.NoError(t, err) + require.NotNil(t, admin) +} + +func TestNewWithAuthProvider(t *testing.T) { + config := &common.Config{} + + tokenAuth, err := auth.NewAuthenticationToken("eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9."+ + "eyJzdWIiOiJhZG1pbiIsImlhdCI6MTUxNjIzOTAyMn0.sVt6cyu3HKd89LcQvZVMNbqT0DTl3FvG9oYbj8hBDqU", nil) + require.NoError(t, err) + require.NotNil(t, tokenAuth) + + admin, err := NewPulsarClientWithAuthProvider(config, tokenAuth) + require.NoError(t, err) + require.NotNil(t, admin) +} + +type customAuthProvider struct { + transport http.RoundTripper +} + +var _ auth.Provider = &customAuthProvider{} + +func (c *customAuthProvider) RoundTrip(req *http.Request) (*http.Response, error) { + panic("implement me") +} + +func (c *customAuthProvider) Transport() http.RoundTripper { + return c.transport +} + +func (c *customAuthProvider) WithTransport(transport http.RoundTripper) { + c.transport = transport +} + +func TestNewWithCustomAuthProviderWithTransport(t *testing.T) { + config := &common.Config{} + defaultTransport, err := auth.NewDefaultTransport(config) + require.NoError(t, err) + + customAuthProvider := &customAuthProvider{ + transport: defaultTransport, + } + + admin, err := NewPulsarClientWithAuthProvider(config, customAuthProvider) + require.NoError(t, err) + require.NotNil(t, admin) + + // Expected the transport of customAuthProvider will not be overwritten. + require.Equal(t, defaultTransport, admin.(*pulsarClient).Client.HTTPClient.Transport) +} + +func TestNewWithTlsAllowInsecure(t *testing.T) { + config := &common.Config{ + TLSAllowInsecureConnection: true, + } + admin, err := New(config) + require.NoError(t, err) + require.NotNil(t, admin) + + pulsarClientS := admin.(*pulsarClient) + require.NotNil(t, pulsarClientS.Client.HTTPClient.Transport) + tr := pulsarClientS.Client.HTTPClient.Transport.(*http.Transport) + require.NotNil(t, tr) + require.NotNil(t, tr.TLSClientConfig) + require.True(t, tr.TLSClientConfig.InsecureSkipVerify) +} diff --git a/pulsaradmin/pkg/pulsar/utils/utils_test.go b/pulsaradmin/pkg/pulsar/utils/utils_test.go index b9ce770aed..df88019db4 100644 --- a/pulsaradmin/pkg/pulsar/utils/utils_test.go +++ b/pulsaradmin/pkg/pulsar/utils/utils_test.go @@ -58,4 +58,10 @@ func TestIsNilFixed(t *testing.T) { var ch chan string assert.True(t, IsNilFixed(ch)) + + var nilInterface People + assert.True(t, IsNilFixed(nilInterface)) + + // pointer to an interface, the IsNilFixed method cannot check this. + assert.False(t, IsNilFixed(&nilInterface)) } From a45fe8ef9ef8ae1e88f5cc600368981b51bde5fb Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 8 Jun 2022 15:37:27 +0800 Subject: [PATCH 223/348] Fix test new with custom auth provider (streamnative/pulsarctl#734) * test: fix TestNewWithCustomAuthProviderWithTransport Signed-off-by: Zixuan Liu * test: fix test image Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/pulsar/admin_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/admin_test.go b/pulsaradmin/pkg/pulsar/admin_test.go index 194b4b520e..4ab4fa413b 100644 --- a/pulsaradmin/pkg/pulsar/admin_test.go +++ b/pulsaradmin/pkg/pulsar/admin_test.go @@ -85,8 +85,8 @@ func TestNewWithCustomAuthProviderWithTransport(t *testing.T) { require.NoError(t, err) require.NotNil(t, admin) - // Expected the transport of customAuthProvider will not be overwritten. - require.Equal(t, defaultTransport, admin.(*pulsarClient).Client.HTTPClient.Transport) + // Expected the customAuthProvider will not be overwritten. + require.Equal(t, customAuthProvider, admin.(*pulsarClient).Client.HTTPClient.Transport) } func TestNewWithTlsAllowInsecure(t *testing.T) { From 353aa4e92ea0f9ae4fa814bf39505582462cb8ef Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 14 Sep 2022 11:55:17 +0800 Subject: [PATCH 224/348] fix: Remove encoding for the topic (streamnative/pulsarctl#828) Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/pulsar/schema.go | 10 +++++----- pulsaradmin/pkg/pulsar/utils/topic_name.go | 2 +- .../pkg/pulsar/utils/topic_name_test.go | 18 ------------------ 3 files changed, 6 insertions(+), 24 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/schema.go b/pulsaradmin/pkg/pulsar/schema.go index 5769bc2abb..fb20dae34f 100644 --- a/pulsaradmin/pkg/pulsar/schema.go +++ b/pulsaradmin/pkg/pulsar/schema.go @@ -62,7 +62,7 @@ func (s *schemas) GetSchemaInfo(topic string) (*utils.SchemaInfo, error) { } var response utils.GetSchemaResponse endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), - topicName.GetEncodedTopic(), "schema") + topicName.GetLocalName(), "schema") err = s.pulsar.Client.Get(endpoint, &response) if err != nil { @@ -80,7 +80,7 @@ func (s *schemas) GetSchemaInfoWithVersion(topic string) (*utils.SchemaInfoWithV } var response utils.GetSchemaResponse endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), - topicName.GetEncodedTopic(), "schema") + topicName.GetLocalName(), "schema") err = s.pulsar.Client.Get(endpoint, &response) if err != nil { @@ -99,7 +99,7 @@ func (s *schemas) GetSchemaInfoByVersion(topic string, version int64) (*utils.Sc } var response utils.GetSchemaResponse - endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetEncodedTopic(), + endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetLocalName(), "schema", strconv.FormatInt(version, 10)) err = s.pulsar.Client.Get(endpoint, &response) @@ -118,7 +118,7 @@ func (s *schemas) DeleteSchema(topic string) error { } endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), - topicName.GetEncodedTopic(), "schema") + topicName.GetLocalName(), "schema") fmt.Println(endpoint) @@ -132,7 +132,7 @@ func (s *schemas) CreateSchemaByPayload(topic string, schemaPayload utils.PostSc } endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), - topicName.GetEncodedTopic(), "schema") + topicName.GetLocalName(), "schema") return s.pulsar.Client.Post(endpoint, &schemaPayload) } diff --git a/pulsaradmin/pkg/pulsar/utils/topic_name.go b/pulsaradmin/pkg/pulsar/utils/topic_name.go index 52c44aae09..268abd73d1 100644 --- a/pulsaradmin/pkg/pulsar/utils/topic_name.go +++ b/pulsaradmin/pkg/pulsar/utils/topic_name.go @@ -119,7 +119,7 @@ func (t *TopicName) IsPersistent() bool { } func (t *TopicName) GetRestPath() string { - return fmt.Sprintf("%s/%s/%s/%s", t.domain, t.tenant, t.namespace, t.GetEncodedTopic()) + return fmt.Sprintf("%s/%s/%s/%s", t.domain, t.tenant, t.namespace, t.topic) } func (t *TopicName) GetEncodedTopic() string { diff --git a/pulsaradmin/pkg/pulsar/utils/topic_name_test.go b/pulsaradmin/pkg/pulsar/utils/topic_name_test.go index 8eba39ce64..27e5002488 100644 --- a/pulsaradmin/pkg/pulsar/utils/topic_name_test.go +++ b/pulsaradmin/pkg/pulsar/utils/topic_name_test.go @@ -18,7 +18,6 @@ package utils import ( - "net/url" "testing" "github.com/stretchr/testify/assert" @@ -74,20 +73,3 @@ func TestGetTopicName(t *testing.T) { assert.Equal(t, "topic name can not be empty", err.Error()) assert.Nil(t, fail) } - -func TestTopicNameEncodeTest(t *testing.T) { - encodedName := "a%3Aen-in_in_business_content_item_20150312173022_https%5C%3A%2F%2Fin.news.example.com%2Fr" - rawName := "a:en-in_in_business_content_item_20150312173022_https\\://in.news.example.com/r" - - assert.Equal(t, encodedName, url.QueryEscape(rawName)) - o, err := url.QueryUnescape(encodedName) - assert.Nil(t, err) - assert.Equal(t, rawName, o) - - topicName, err := GetTopicName("persistent://prop/ns/" + rawName) - assert.Nil(t, err) - - assert.Equal(t, rawName, topicName.topic) - assert.Equal(t, encodedName, topicName.GetEncodedTopic()) - assert.Equal(t, "persistent/prop/ns/"+encodedName, topicName.GetRestPath()) -} From 544789ed29eafb63dad494900407fc7cd15cc603 Mon Sep 17 00:00:00 2001 From: Guangning E Date: Mon, 10 Oct 2022 11:26:22 +0800 Subject: [PATCH 225/348] Update pulsarctl go version (streamnative/pulsarctl#853) --- pulsaradmin/go.mod | 45 ++++++++++++++++++++------------------------- 1 file changed, 20 insertions(+), 25 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index a187606338..34b6de41fc 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -3,8 +3,8 @@ module github.com/streamnative/pulsar-admin-go go 1.18 require ( - github.com/99designs/keyring v1.1.6 - github.com/apache/pulsar-client-go/oauth2 v0.0.0-20211108044248-fe3b7c4e445b + github.com/99designs/keyring v1.2.1 + github.com/apache/pulsar-client-go v0.9.0 github.com/docker/go-connections v0.4.0 github.com/fatih/color v1.7.0 github.com/form3tech-oss/jwt-go v3.2.3+incompatible @@ -13,61 +13,56 @@ require ( github.com/imdario/mergo v0.3.8 github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b - github.com/magiconair/properties v1.8.0 + github.com/magiconair/properties v1.8.5 github.com/olekukonko/tablewriter v0.0.1 - github.com/onsi/gomega v1.18.0 + github.com/onsi/gomega v1.19.0 github.com/pkg/errors v0.9.1 - github.com/spf13/cobra v0.0.5 + github.com/spf13/cobra v1.2.1 github.com/spf13/pflag v1.0.5 - github.com/stretchr/testify v1.5.1 + github.com/stretchr/testify v1.8.0 github.com/testcontainers/testcontainers-go v0.0.10 - golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93 + golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 gopkg.in/yaml.v2 v2.4.0 ) require ( + github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect github.com/Microsoft/go-winio v0.4.11 // indirect github.com/Microsoft/hcsshim v0.8.6 // indirect github.com/cenkalti/backoff v2.2.1+incompatible // indirect github.com/containerd/continuity v0.0.0-20190426062206-aaeac12a7ffc // indirect - github.com/danieljoos/wincred v1.0.2 // indirect + github.com/danieljoos/wincred v1.1.2 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/docker/distribution v2.7.1-0.20190205005809-0d3efadf0154+incompatible // indirect github.com/docker/docker v0.7.3-0.20190506211059-b20a14b54661 // indirect github.com/docker/go-units v0.3.3 // indirect - github.com/dvsekhvalnov/jose2go v0.0.0-20200901110807-248326c1351b // indirect - github.com/go-sql-driver/mysql v1.5.0 // indirect + github.com/dvsekhvalnov/jose2go v1.5.0 // indirect github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect github.com/gogo/protobuf v1.3.2 // indirect - github.com/gorilla/mux v1.7.4 // indirect + github.com/golang-jwt/jwt v3.2.1+incompatible // indirect github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/inconshreveable/mousetrap v1.0.0 // indirect - github.com/keybase/go-keychain v0.0.0-20190712205309-48d3d31d256d // indirect - github.com/konsorten/go-windows-terminal-sequences v1.0.1 // indirect - github.com/kr/pretty v0.2.0 // indirect + github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect github.com/mattn/go-colorable v0.1.2 // indirect github.com/mattn/go-isatty v0.0.8 // indirect github.com/mattn/go-runewidth v0.0.4 // indirect - github.com/mitchellh/go-homedir v1.1.0 // indirect github.com/mtibben/percent v0.2.1 // indirect github.com/opencontainers/go-digest v1.0.0-rc1 // indirect github.com/opencontainers/image-spec v1.0.1 // indirect github.com/opencontainers/runc v0.1.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/satori/go.uuid v1.2.0 // indirect - github.com/sirupsen/logrus v1.4.2 // indirect - golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 // indirect - golang.org/x/net v0.0.0-20210428140749-89ef3d95e781 // indirect - golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9 // indirect + github.com/sirupsen/logrus v1.6.0 // indirect + golang.org/x/net v0.0.0-20220225172249-27dd8689420f // indirect + golang.org/x/sync v0.0.0-20210220032951-036812b2e83c // indirect golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 // indirect - golang.org/x/text v0.3.6 // indirect + golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 // indirect + golang.org/x/text v0.3.7 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto v0.0.0-20200825200019-8632dd797987 // indirect - google.golang.org/grpc v1.31.0 // indirect + google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c // indirect + google.golang.org/grpc v1.38.0 // indirect google.golang.org/protobuf v1.26.0 // indirect - gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect ) -replace github.com/apache/pulsar-client-go/oauth2 => github.com/apache/pulsar-client-go/oauth2 v0.0.0-20211006154457-742f1b107403 - replace golang.org/x/sys => golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 From e39a7e176cb39b4dc1d122614f89fa27c8f3a3de Mon Sep 17 00:00:00 2001 From: Will Nyffenegger Date: Tue, 18 Oct 2022 09:38:45 -0500 Subject: [PATCH 226/348] add fields to topic stats (streamnative/pulsarctl#858) --- pulsaradmin/pkg/pulsar/utils/data.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index 4941456dd8..7207dad7b9 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -183,6 +183,8 @@ type NamespacesData struct { } type TopicStats struct { + MsgCounterIn int64 `json:"msgInCounter"` + MsgCounterOut int64 `json:"msgOutCounter"` MsgRateIn float64 `json:"msgRateIn"` MsgRateOut float64 `json:"msgRateOut"` MsgThroughputIn float64 `json:"msgThroughputIn"` @@ -206,11 +208,15 @@ type PublisherStats struct { type SubscriptionStats struct { BlockedSubscriptionOnUnackedMsgs bool `json:"blockedSubscriptionOnUnackedMsgs"` IsReplicated bool `json:"isReplicated"` + LastConsumedFlowTimestamp int64 `json:"lastConsumedFlowTimestamp"` + LastConsumedTimestamp int64 `json:"lastConsumedTimestamp"` + LastAckedTimestamp int64 `json:"lastAckedTimestamp"` MsgRateOut float64 `json:"msgRateOut"` MsgThroughputOut float64 `json:"msgThroughputOut"` MsgRateRedeliver float64 `json:"msgRateRedeliver"` MsgRateExpired float64 `json:"msgRateExpired"` MsgBacklog int64 `json:"msgBacklog"` + MsgBacklogNoDelayed int64 `json:"msgBacklogNoDelayed"` MsgDelayed int64 `json:"msgDelayed"` UnAckedMessages int64 `json:"unackedMessages"` SubType string `json:"type"` From c382d67a3fc0e98e1716f02b81917a03afe490de Mon Sep 17 00:00:00 2001 From: Will Nyffenegger Date: Tue, 18 Oct 2022 22:38:50 -0500 Subject: [PATCH 227/348] add estimated backlog size as measure of unconsumed messages (streamnative/pulsarctl#860) --- pulsaradmin/pkg/pulsar/utils/data.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index 7207dad7b9..f581e964ca 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -183,6 +183,7 @@ type NamespacesData struct { } type TopicStats struct { + BacklogSize int64 `json:"backlogSize"` MsgCounterIn int64 `json:"msgInCounter"` MsgCounterOut int64 `json:"msgOutCounter"` MsgRateIn float64 `json:"msgRateIn"` From b0a5862c52bf984638cba4d848d66c469308c3d1 Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Mon, 7 Nov 2022 17:37:13 +0800 Subject: [PATCH 228/348] support oauth2 with auth params (streamnative/pulsarctl#876) * support oauth2 with auth params * goimport * fix style * fix style * Update pkg/auth/oauth2.go Co-authored-by: Zixuan Liu * Update pkg/auth/oauth2.go Co-authored-by: Zixuan Liu * Update pkg/auth/oauth2.go Co-authored-by: Zixuan Liu * Update pkg/auth/oauth2.go Co-authored-by: Zixuan Liu Co-authored-by: Zixuan Liu --- pulsaradmin/pkg/auth/auth_provider.go | 4 ++++ pulsaradmin/pkg/auth/oauth2.go | 26 ++++++++++++++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/auth/auth_provider.go index 17bb69e06b..3bcf8675b1 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/auth/auth_provider.go @@ -52,6 +52,10 @@ func GetAuthProvider(config *common.Config) (Provider, error) { fallthrough case TokePluginShortName: provider, err = NewAuthenticationTokenFromAuthParams(config.AuthParams, defaultTransport) + case OAuth2PluginName: + fallthrough + case OAuth2PluginShortName: + provider, err = NewAuthenticationOAuth2FromAuthParams(config.AuthParams, defaultTransport) default: switch { case len(config.TLSCertFile) > 0 && len(config.TLSKeyFile) > 0: diff --git a/pulsaradmin/pkg/auth/oauth2.go b/pulsaradmin/pkg/auth/oauth2.go index 1e50adb9e2..09f8362703 100644 --- a/pulsaradmin/pkg/auth/oauth2.go +++ b/pulsaradmin/pkg/auth/oauth2.go @@ -18,6 +18,7 @@ package auth import ( + "encoding/json" "net/http" "path/filepath" @@ -32,6 +33,19 @@ import ( xoauth2 "golang.org/x/oauth2" ) +const ( + OAuth2PluginName = "org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2" + OAuth2PluginShortName = "oauth2" +) + +type OAuth2ClientCredentials struct { + IssuerURL string `json:"issuerUrl,omitempty"` + Audience string `json:"audience,omitempty"` + Scope string `json:"scope,omitempty"` + PrivateKey string `json:"privateKey,omitempty"` + ClientID string `json:"clientId,omitempty"` +} + type OAuth2Provider struct { clock clock2.RealClock issuer oauth2.Issuer @@ -85,6 +99,18 @@ func NewAuthenticationOAuth2WithDefaultFlow(issuer oauth2.Issuer, keyFile string return p, p.loadGrant() } +func NewAuthenticationOAuth2FromAuthParams(encodedAuthParam string, + transport http.RoundTripper) (*OAuth2Provider, error) { + + var paramsJSON OAuth2ClientCredentials + err := json.Unmarshal([]byte(encodedAuthParam), ¶msJSON) + if err != nil { + return nil, err + } + return NewAuthenticationOAuth2WithParams(paramsJSON.IssuerURL, paramsJSON.ClientID, paramsJSON.Audience, + paramsJSON.Scope, transport) +} + func NewAuthenticationOAuth2WithParams( issuerEndpoint, clientID, From 1cbd268df8fdc08a0fc654d1769cdf43ec42cb21 Mon Sep 17 00:00:00 2001 From: Asher Goldberg <43661200+asher-goldberg@users.noreply.github.com> Date: Thu, 10 Nov 2022 23:20:50 -0500 Subject: [PATCH 229/348] update resource keys to match comments (streamnative/pulsarctl#879) --- pulsaradmin/pkg/pulsar/utils/resources.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/utils/resources.go b/pulsaradmin/pkg/pulsar/utils/resources.go index d5b7f3f074..a4f3ddbcbe 100644 --- a/pulsaradmin/pkg/pulsar/utils/resources.go +++ b/pulsaradmin/pkg/pulsar/utils/resources.go @@ -28,9 +28,9 @@ func NewDefaultResources() *Resources { // Default cpu is 1 core CPU: 1, // Default memory is 1GB - Disk: 1073741824, + RAM: 1073741824, // Default disk is 10GB - RAM: 10737418240, + Disk: 10737418240, } return resources From aab8634f00b646559d8d270dfe5cdb4e526b4519 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 11 Nov 2022 18:16:19 +0800 Subject: [PATCH 230/348] Add get-message-by-id command (streamnative/pulsarctl#882) --- pulsaradmin/pkg/pulsar/subscription.go | 25 +++++++++++++++++++++++++ pulsaradmin/pkg/pulsar/utils/message.go | 4 +++- 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/pulsar/subscription.go index 42d1be8c39..ede71b06ee 100644 --- a/pulsaradmin/pkg/pulsar/subscription.go +++ b/pulsaradmin/pkg/pulsar/subscription.go @@ -69,6 +69,9 @@ type Subscriptions interface { // PeekMessages peeks messages from a topic subscription PeekMessages(utils.TopicName, string, int) ([]*utils.Message, error) + + // GetMessageByID gets message by its ledgerID and entryID + GetMessageByID(topic utils.TopicName, ledgerID, entryID int64) (*utils.Message, error) } type subscriptions struct { @@ -171,6 +174,28 @@ func (s *subscriptions) peekNthMessage(topic utils.TopicName, sName string, pos return handleResp(topic, resp) } +func (s *subscriptions) GetMessageByID(topic utils.TopicName, ledgerID, entryID int64) (*utils.Message, error) { + ledgerIDStr := strconv.FormatInt(ledgerID, 10) + entryIDStr := strconv.FormatInt(entryID, 10) + + endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), "ledger", ledgerIDStr, "entry", entryIDStr) + resp, err := s.pulsar.Client.MakeRequest(http.MethodGet, endpoint) + if err != nil { + return nil, err + } + defer safeRespClose(resp) + + messages, err := handleResp(topic, resp) + if err != nil { + return nil, err + } + + if len(messages) == 0 { + return nil, nil + } + return messages[0], nil +} + // safeRespClose is used to close a response body func safeRespClose(resp *http.Response) { if resp != nil { diff --git a/pulsaradmin/pkg/pulsar/utils/message.go b/pulsaradmin/pkg/pulsar/utils/message.go index f60bbff66a..6ab1a5cfa4 100644 --- a/pulsaradmin/pkg/pulsar/utils/message.go +++ b/pulsaradmin/pkg/pulsar/utils/message.go @@ -18,7 +18,9 @@ package utils // nolint -import "github.com/golang/protobuf/proto" +import ( + "github.com/golang/protobuf/proto" +) type Message struct { MessageID MessageID From d0ffd40178113710ddfc14c5286de1dc8bd6507c Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 12 Dec 2022 10:34:21 +0800 Subject: [PATCH 231/348] fix: add miss fields to PersistentTopicInternalStats (streamnative/pulsarctl#477) Signed-off-by: Zixuan Liu Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/pulsar/utils/data.go | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index f581e964ca..8908343fd7 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -265,6 +265,8 @@ type PersistentTopicInternalStats struct { State string `json:"state"` Ledgers []LedgerInfo `json:"ledgers"` Cursors map[string]CursorStats `json:"cursors"` + SchemaLedgers []SchemaLedger `json:"schemaLedgers"` + CompactedLedger CompactedLedger `json:"compactedLedger"` } type LedgerInfo struct { @@ -402,3 +404,19 @@ type PublishRateData struct { PublishThrottlingRateInMsg int64 `json:"publishThrottlingRateInMsg"` PublishThrottlingRateInByte int64 `json:"publishThrottlingRateInByte"` } + +type SchemaLedger struct { + LedgerID int64 `json:"ledgerId"` + Entries int64 `json:"entries"` + Size int64 `json:"size"` + Timestamp int64 `json:"timestamp"` + IsOffloaded bool `json:"isOffloaded"` +} + +type CompactedLedger struct { + LedgerID int64 `json:"ledgerId"` + Entries int64 `json:"entries"` + Size int64 `json:"size"` + Offloaded bool `json:"offloaded"` + UnderReplicated bool `json:"underReplicated"` +} From c6f2db8bea792978b2b9707fb68e237cf7992973 Mon Sep 17 00:00:00 2001 From: Guangning E Date: Mon, 12 Dec 2022 11:45:09 +0800 Subject: [PATCH 232/348] Upgrade jwt package, support custom claim and header (streamnative/pulsarctl#906) * Upgrade jwt package Support custom claim and header * Fixed go import * Remove golang 13, 14, 15 ci check --- pulsaradmin/go.mod | 2 +- pulsaradmin/pkg/pulsar/token.go | 39 ++++++++++++++++++++++++--------- 2 files changed, 30 insertions(+), 11 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index 34b6de41fc..ae0fe54ecb 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -7,8 +7,8 @@ require ( github.com/apache/pulsar-client-go v0.9.0 github.com/docker/go-connections v0.4.0 github.com/fatih/color v1.7.0 - github.com/form3tech-oss/jwt-go v3.2.3+incompatible github.com/ghodss/yaml v1.0.0 + github.com/golang-jwt/jwt/v4 v4.4.3 github.com/golang/protobuf v1.5.2 github.com/imdario/mergo v0.3.8 github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 diff --git a/pulsaradmin/pkg/pulsar/token.go b/pulsaradmin/pkg/pulsar/token.go index 1c2db44495..9d4fdbccb6 100644 --- a/pulsaradmin/pkg/pulsar/token.go +++ b/pulsaradmin/pkg/pulsar/token.go @@ -18,12 +18,15 @@ package pulsar import ( + "encoding/base64" "strings" + "time" + + "github.com/golang-jwt/jwt/v4" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/algorithm" "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" - "github.com/form3tech-oss/jwt-go" "github.com/pkg/errors" ) @@ -38,6 +41,10 @@ type Token interface { // object and the expire time Create(algorithm.Algorithm, interface{}, string, int64) (string, error) + // CreateToken creates a token object using the specified signature algorithm, private key + // custom claim and header + CreateToken(algorithm.Algorithm, interface{}, *jwt.MapClaims, map[string]interface{}) (string, error) + // Validate a token is valid or not Validate(algorithm.Algorithm, string, interface{}) (string, int64, error) @@ -77,13 +84,25 @@ func (t *token) CreateSecretKey(signatureAlgorithm algorithm.Algorithm) ([]byte, func (t *token) Create(algorithm algorithm.Algorithm, signKey interface{}, subject string, expireTime int64) (string, error) { - claims := &jwt.StandardClaims{ - Subject: subject, - ExpiresAt: expireTime, + claims := &jwt.MapClaims{ + "sub": subject, + "exp": jwt.NewNumericDate(time.Unix(expireTime, 0)), } - signMethod := parseAlgorithmToJwtSignMethod(algorithm) - tokenString := jwt.NewWithClaims(signMethod, claims) + return t.CreateToken(algorithm, signKey, claims, nil) +} +func (t *token) CreateToken( + algorithm algorithm.Algorithm, + signKey interface{}, + mapClaims *jwt.MapClaims, + headers map[string]interface{}) (string, error) { + signMethod := parseAlgorithmToJwtSignMethod(algorithm) + tokenString := jwt.NewWithClaims(signMethod, mapClaims) + if headers != nil && len(headers) > 0 { + for s, i := range headers { + tokenString.Header[s] = i + } + } return tokenString.SignedString(signKey) } @@ -110,20 +129,20 @@ func (t *token) Validate(algorithm algorithm.Algorithm, tokenString string, func (t *token) GetAlgorithm(tokenString string) (string, error) { parts := strings.Split(tokenString, ".") - algorithm, err := jwt.DecodeSegment(parts[0]) + alg, err := base64.RawURLEncoding.DecodeString(parts[0]) if err != nil { return "", err } - return string(algorithm), nil + return string(alg), nil } func (t *token) GetSubject(tokenString string) (string, error) { parts := strings.Split(tokenString, ".") - algorithm, err := jwt.DecodeSegment(parts[1]) + alg, err := base64.RawURLEncoding.DecodeString(parts[1]) if err != nil { return "", err } - return string(algorithm), nil + return string(alg), nil } func parseAlgorithmToJwtSignMethod(a algorithm.Algorithm) jwt.SigningMethod { From 56cbec9c355262add34304d79765291e5fed7208 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 12 Dec 2022 12:10:55 +0800 Subject: [PATCH 233/348] Fix the wrong field of the internal stats and internal info (streamnative/pulsarctl#907) --- pulsaradmin/pkg/pulsar/utils/data.go | 33 ++++++++++++++++++---------- 1 file changed, 22 insertions(+), 11 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index 8908343fd7..2ac50b577c 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -151,13 +151,22 @@ type PartitionedTopicMetadata struct { Partitions int `json:"partitions"` } +type ManagedLedgerInfoLedgerInfo struct { + LedgerID int64 `json:"ledgerId"` + Entries int64 `json:"entries"` + Size int64 `json:"size"` + Timestamp int64 `json:"timestamp"` + Offloaded bool `json:"isOffloaded"` + OffloadedContextUUID string `json:"offloadedContextUuid"` +} + type ManagedLedgerInfo struct { - Version int `json:"version"` - CreationDate string `json:"creationDate"` - ModificationData string `json:"modificationData"` - Ledgers []LedgerInfo `json:"ledgers"` - TerminatedPosition PositionInfo `json:"terminatedPosition"` - Cursors map[string]CursorInfo `json:"cursors"` + Version int `json:"version"` + CreationDate string `json:"creationDate"` + ModificationData string `json:"modificationData"` + Ledgers []ManagedLedgerInfoLedgerInfo `json:"ledgers"` + TerminatedPosition PositionInfo `json:"terminatedPosition"` + Cursors map[string]CursorInfo `json:"cursors"` } type NamespacesData struct { @@ -270,11 +279,13 @@ type PersistentTopicInternalStats struct { } type LedgerInfo struct { - LedgerID int64 `json:"ledgerId"` - Entries int64 `json:"entries"` - Size int64 `json:"size"` - Timestamp int64 `json:"timestamp"` - Offloaded bool `json:"isOffloaded"` + LedgerID int64 `json:"ledgerId"` + Entries int64 `json:"entries"` + Size int64 `json:"size"` + Timestamp int64 `json:"timestamp"` + Offloaded bool `json:"offloaded"` + MetaData string `json:"metadata"` + UnderReplicated bool `json:"underReplicated"` } type CursorInfo struct { From 68dc725ae8673d6a03c71ce5de82b7ed362cc295 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 12 Dec 2022 12:12:22 +0800 Subject: [PATCH 234/348] feat: support delete a subscription forcefully (streamnative/pulsarctl#609) Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/pulsar/subscription.go | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/pulsar/subscription.go index ede71b06ee..a17b381bd0 100644 --- a/pulsaradmin/pkg/pulsar/subscription.go +++ b/pulsaradmin/pkg/pulsar/subscription.go @@ -41,6 +41,9 @@ type Subscriptions interface { // Delete a persistent subscription from a topic. There should not be any active consumers on the subscription Delete(utils.TopicName, string) error + // ForceDelete deletes a subscription forcefully + ForceDelete(utils.TopicName, string) error + // List returns the list of subscriptions List(utils.TopicName) ([]string, error) @@ -94,9 +97,19 @@ func (s *subscriptions) Create(topic utils.TopicName, sName string, messageID ut return s.pulsar.Client.Put(endpoint, messageID) } +func (s *subscriptions) delete(topic utils.TopicName, subName string, force bool) error { + endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(subName)) + queryParams := make(map[string]string) + queryParams["force"] = strconv.FormatBool(force) + return s.pulsar.Client.DeleteWithQueryParams(endpoint, queryParams) +} + func (s *subscriptions) Delete(topic utils.TopicName, sName string) error { - endpoint := s.pulsar.endpoint(s.basePath, topic.GetRestPath(), s.SubPath, url.PathEscape(sName)) - return s.pulsar.Client.Delete(endpoint) + return s.delete(topic, sName, false) +} + +func (s *subscriptions) ForceDelete(topic utils.TopicName, sName string) error { + return s.delete(topic, sName, true) } func (s *subscriptions) List(topic utils.TopicName) ([]string, error) { From a05b06eb22010e6b98eefff29cd2aa17609296b2 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 31 Jan 2023 11:52:57 +0800 Subject: [PATCH 235/348] fix: fix token exp (streamnative/pulsarctl#976) * fix: fix token exp Signed-off-by: Zixuan Liu * Improve condition Signed-off-by: Zixuan Liu --------- Signed-off-by: Zixuan Liu --- pulsaradmin/pkg/pulsar/token.go | 25 +++++++++--- pulsaradmin/pkg/pulsar/token_test.go | 61 ++++++++++++++++++++++++++++ 2 files changed, 80 insertions(+), 6 deletions(-) create mode 100644 pulsaradmin/pkg/pulsar/token_test.go diff --git a/pulsaradmin/pkg/pulsar/token.go b/pulsaradmin/pkg/pulsar/token.go index 9d4fdbccb6..c9263b94ec 100644 --- a/pulsaradmin/pkg/pulsar/token.go +++ b/pulsaradmin/pkg/pulsar/token.go @@ -84,10 +84,18 @@ func (t *token) CreateSecretKey(signatureAlgorithm algorithm.Algorithm) ([]byte, func (t *token) Create(algorithm algorithm.Algorithm, signKey interface{}, subject string, expireTime int64) (string, error) { - claims := &jwt.MapClaims{ - "sub": subject, - "exp": jwt.NewNumericDate(time.Unix(expireTime, 0)), + var claims *jwt.MapClaims + if expireTime <= 0 { + claims = &jwt.MapClaims{ + "sub": subject, + } + } else { + claims = &jwt.MapClaims{ + "sub": subject, + "exp": jwt.NewNumericDate(time.Unix(expireTime, 0)), + } } + return t.CreateToken(algorithm, signKey, claims, nil) } @@ -110,7 +118,7 @@ func (t *token) Validate(algorithm algorithm.Algorithm, tokenString string, signKey interface{}) (string, int64, error) { // verify the signature algorithm - parsedToken, err := jwt.ParseWithClaims(tokenString, &jwt.StandardClaims{}, + parsedToken, err := jwt.ParseWithClaims(tokenString, &jwt.RegisteredClaims{}, func(jt *jwt.Token) (i interface{}, e error) { signMethod := parseAlgorithmToJwtSignMethod(algorithm) if jt.Method != signMethod { @@ -120,8 +128,13 @@ func (t *token) Validate(algorithm algorithm.Algorithm, tokenString string, }) // get the subject and the expire time - if claim, ok := parsedToken.Claims.(*jwt.StandardClaims); parsedToken.Valid && ok { - return claim.Subject, claim.ExpiresAt, nil + if claim, ok := parsedToken.Claims.(*jwt.RegisteredClaims); parsedToken.Valid && ok { + expiresAt := claim.ExpiresAt + exp := int64(0) + if expiresAt != nil { + exp = expiresAt.Unix() + } + return claim.Subject, exp, nil } return "", 0, err diff --git a/pulsaradmin/pkg/pulsar/token_test.go b/pulsaradmin/pkg/pulsar/token_test.go new file mode 100644 index 0000000000..b30af6dab4 --- /dev/null +++ b/pulsaradmin/pkg/pulsar/token_test.go @@ -0,0 +1,61 @@ +// 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 pulsar + +import ( + "testing" + "time" + + "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/algorithm" + "github.com/stretchr/testify/require" +) + +func TestCreateToken(t *testing.T) { + tokenProvider := token{} + + alg := algorithm.HS256 + key, err := tokenProvider.CreateSecretKey(alg) + require.NoError(t, err) + + subject := "test-role" + myToken, err := tokenProvider.Create(alg, key, subject, 0) + require.NoError(t, err) + + parsedSubject, exp, err := tokenProvider.Validate(alg, myToken, key) + require.NoError(t, err) + require.Equal(t, subject, parsedSubject) + require.Equal(t, exp, int64(0)) +} + +func TestCreateTokenWithExp(t *testing.T) { + tokenProvider := token{} + + alg := algorithm.HS256 + key, err := tokenProvider.CreateSecretKey(alg) + require.NoError(t, err) + + subject := "test-role" + exp := time.Now().Add(time.Hour).Unix() + myToken, err := tokenProvider.Create(alg, key, subject, exp) + require.NoError(t, err) + + parsedSubject, exp, err := tokenProvider.Validate(alg, myToken, key) + require.NoError(t, err) + require.Equal(t, subject, parsedSubject) + require.Equal(t, exp, exp) +} From 588ecb12a5c46d7cf97058fa0d77be49ab54fdce Mon Sep 17 00:00:00 2001 From: Fushu Wang Date: Fri, 17 Feb 2023 15:45:51 +0800 Subject: [PATCH 236/348] Add auth parameter to cluster management (streamnative/pulsarctl#986) * feat: add parameters authenticationPlugin and authenticationParameters for cluster management * feat: add new api SetReplicationClusters and GetReplicationClusters for topic --- pulsaradmin/pkg/pulsar/topic.go | 18 ++++++++++++++++++ pulsaradmin/pkg/pulsar/utils/data.go | 14 ++++++++------ 2 files changed, 26 insertions(+), 6 deletions(-) diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/pulsar/topic.go index b0f0058bcc..e8cdcd9643 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/pulsar/topic.go @@ -224,6 +224,12 @@ type Topics interface { // SetInactiveTopicPolicies sets the inactive topic policies on a topic SetInactiveTopicPolicies(topic utils.TopicName, data utils.InactiveTopicPolicies) error + + // GetReplicationClusters get the replication clusters of a topic + GetReplicationClusters(topic utils.TopicName) ([]string, error) + + // SetReplicationClusters sets the replication clusters on a topic + SetReplicationClusters(topic utils.TopicName, data []string) error } type topics struct { @@ -696,3 +702,15 @@ func (t *topics) SetInactiveTopicPolicies(topic utils.TopicName, data utils.Inac endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "inactiveTopicPolicies") return t.pulsar.Client.Post(endpoint, data) } + +func (t *topics) SetReplicationClusters(topic utils.TopicName, data []string) error { + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "replication") + return t.pulsar.Client.Post(endpoint, data) +} + +func (t *topics) GetReplicationClusters(topic utils.TopicName) ([]string, error) { + var data []string + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "replication") + err := t.pulsar.Client.Get(endpoint, &data) + return data, err +} diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/pulsar/utils/data.go index 2ac50b577c..db0eeb32d6 100644 --- a/pulsaradmin/pkg/pulsar/utils/data.go +++ b/pulsaradmin/pkg/pulsar/utils/data.go @@ -19,12 +19,14 @@ package utils // ClusterData information on a cluster type ClusterData struct { - Name string `json:"-"` - ServiceURL string `json:"serviceUrl"` - ServiceURLTls string `json:"serviceUrlTls"` - BrokerServiceURL string `json:"brokerServiceUrl"` - BrokerServiceURLTls string `json:"brokerServiceUrlTls"` - PeerClusterNames []string `json:"peerClusterNames"` + Name string `json:"-"` + ServiceURL string `json:"serviceUrl"` + ServiceURLTls string `json:"serviceUrlTls"` + BrokerServiceURL string `json:"brokerServiceUrl"` + BrokerServiceURLTls string `json:"brokerServiceUrlTls"` + PeerClusterNames []string `json:"peerClusterNames"` + AuthenticationPlugin string `json:"authenticationPlugin"` + AuthenticationParameters string `json:"authenticationParameters"` } // FunctionData information for a Pulsar Function From 2a2af482a3bfac07f66eb3fc599870ca33fcb3e7 Mon Sep 17 00:00:00 2001 From: Max Xu Date: Wed, 1 Mar 2023 15:31:33 +0800 Subject: [PATCH 237/348] chore: go mod tidy Signed-off-by: Max Xu --- pulsaradmin/go.mod | 35 --- pulsaradmin/go.sum | 682 +++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 682 insertions(+), 35 deletions(-) create mode 100644 pulsaradmin/go.sum diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index ae0fe54ecb..f1a1bbd0e2 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -5,62 +5,27 @@ go 1.18 require ( github.com/99designs/keyring v1.2.1 github.com/apache/pulsar-client-go v0.9.0 - github.com/docker/go-connections v0.4.0 - github.com/fatih/color v1.7.0 - github.com/ghodss/yaml v1.0.0 github.com/golang-jwt/jwt/v4 v4.4.3 github.com/golang/protobuf v1.5.2 - github.com/imdario/mergo v0.3.8 - github.com/kris-nova/logger v0.0.0-20181127235838-fd0d87064b06 - github.com/kris-nova/lolgopher v0.0.0-20180921204813-313b3abb0d9b - github.com/magiconair/properties v1.8.5 - github.com/olekukonko/tablewriter v0.0.1 - github.com/onsi/gomega v1.19.0 github.com/pkg/errors v0.9.1 - github.com/spf13/cobra v1.2.1 - github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.8.0 - github.com/testcontainers/testcontainers-go v0.0.10 golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 - gopkg.in/yaml.v2 v2.4.0 ) require ( github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect - github.com/Microsoft/go-winio v0.4.11 // indirect - github.com/Microsoft/hcsshim v0.8.6 // indirect - github.com/cenkalti/backoff v2.2.1+incompatible // indirect - github.com/containerd/continuity v0.0.0-20190426062206-aaeac12a7ffc // indirect github.com/danieljoos/wincred v1.1.2 // indirect github.com/davecgh/go-spew v1.1.1 // indirect - github.com/docker/distribution v2.7.1-0.20190205005809-0d3efadf0154+incompatible // indirect - github.com/docker/docker v0.7.3-0.20190506211059-b20a14b54661 // indirect - github.com/docker/go-units v0.3.3 // indirect github.com/dvsekhvalnov/jose2go v1.5.0 // indirect github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect - github.com/gogo/protobuf v1.3.2 // indirect github.com/golang-jwt/jwt v3.2.1+incompatible // indirect github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect - github.com/inconshreveable/mousetrap v1.0.0 // indirect - github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect - github.com/mattn/go-colorable v0.1.2 // indirect - github.com/mattn/go-isatty v0.0.8 // indirect - github.com/mattn/go-runewidth v0.0.4 // indirect github.com/mtibben/percent v0.2.1 // indirect - github.com/opencontainers/go-digest v1.0.0-rc1 // indirect - github.com/opencontainers/image-spec v1.0.1 // indirect - github.com/opencontainers/runc v0.1.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/satori/go.uuid v1.2.0 // indirect - github.com/sirupsen/logrus v1.6.0 // indirect golang.org/x/net v0.0.0-20220225172249-27dd8689420f // indirect - golang.org/x/sync v0.0.0-20210220032951-036812b2e83c // indirect golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 // indirect golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 // indirect - golang.org/x/text v0.3.7 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c // indirect - google.golang.org/grpc v1.38.0 // indirect google.golang.org/protobuf v1.26.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/pulsaradmin/go.sum b/pulsaradmin/go.sum new file mode 100644 index 0000000000..36562eafb7 --- /dev/null +++ b/pulsaradmin/go.sum @@ -0,0 +1,682 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= +cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= +cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= +cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= +cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= +cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= +cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= +cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= +cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bPc= +cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= +cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= +cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= +cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= +cloud.google.com/go v0.72.0/go.mod h1:M+5Vjvlc2wnp6tjzE102Dw08nGShTscUx2nZMufOKPI= +cloud.google.com/go v0.74.0/go.mod h1:VV1xSbzvo+9QJOxLDaJfTjx5e+MePCpCWwvftOeQmWk= +cloud.google.com/go v0.78.0/go.mod h1:QjdrLG0uq+YwhjoVOLsS1t7TW8fs36kLs4XO5R5ECHg= +cloud.google.com/go v0.79.0/go.mod h1:3bzgcEeQlzbuEAYu4mrWhKqWjmpprinYgKJLgKHnbb8= +cloud.google.com/go v0.81.0/go.mod h1:mk/AM35KwGk/Nm2YSeZbxXdrNK3KZOYHmLkOqC2V6E0= +cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= +cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= +cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= +cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= +cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= +cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= +cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= +cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= +cloud.google.com/go/firestore v1.1.0/go.mod h1:ulACoGHTpvq5r8rxGJ4ddJZBZqakUQqClKRT5SZwBmk= +cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= +cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= +cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= +cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= +cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= +cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= +cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= +cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= +cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 h1:/vQbFIOMbk2FiG/kXiLl8BRyzTWDw7gX/Hz7Dd5eDMs= +github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4/go.mod h1:hN7oaIRCjzsZ2dE+yG5k+rsdt3qcwykqK6HVGcKwsw4= +github.com/99designs/keyring v1.2.1 h1:tYLp1ULvO7i3fI5vE21ReQuj99QFSs7lGm0xWyJo87o= +github.com/99designs/keyring v1.2.1/go.mod h1:fc+wB5KTk9wQ9sDx0kFXB3A0MaeGHM9AwRStKOQ5vOA= +github.com/AthenZ/athenz v1.10.39/go.mod h1:3Tg8HLsiQZp81BJY58JBeU2BR6B/H4/0MQGfCwhHNEA= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/DataDog/zstd v1.5.0/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= +github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= +github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= +github.com/apache/pulsar-client-go v0.9.0 h1:L5jvGFXJm0JNA/PgUiJctTVHHttCe4wIEFDv4vojiQM= +github.com/apache/pulsar-client-go v0.9.0/go.mod h1:fSAcBipgz4KQ/VgwZEJtQ71cCXMKm8ezznstrozrngw= +github.com/ardielle/ardielle-go v1.5.2/go.mod h1:I4hy1n795cUhaVt/ojz83SNVCYIGsAFAONtv2Dr7HUI= +github.com/ardielle/ardielle-tools v1.5.4/go.mod h1:oZN+JRMnqGiIhrzkRN9l26Cej9dEx4jeNG6A+AdkShk= +github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= +github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= +github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= +github.com/aws/aws-sdk-go v1.32.6/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= +github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= +github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= +github.com/bketelsen/crypt v0.0.4/go.mod h1:aI6NrJ0pMGgvZKL1iVgXLnfIFJtfV+bKCoqOes/6LfM= +github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b/go.mod h1:ac9efd0D1fsDb3EJvhqgXRbFx7bs2wqZ10HQPeU8U/Q= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= +github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= +github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= +github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= +github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= +github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= +github.com/danieljoos/wincred v1.1.2 h1:QLdCxFs1/Yl4zduvBdcHB8goaYk9RARS2SgLLRuAyr0= +github.com/danieljoos/wincred v1.1.2/go.mod h1:GijpziifJoIBfYh+S7BbkdUTU4LfM+QnGqR5Vl2tAx0= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dimfeld/httptreemux v5.0.1+incompatible/go.mod h1:rbUlSV+CCpv/SuqUTP/8Bk2O3LyUV436/yaRGkhP6Z0= +github.com/dvsekhvalnov/jose2go v1.5.0 h1:3j8ya4Z4kMCwT5nXIKFSV84YS+HdqSSO0VsTQxaLAeM= +github.com/dvsekhvalnov/jose2go v1.5.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/go-control-plane v0.9.7/go.mod h1:cwu0lG7PUMfa9snN8LXBig5ynNVH9qI8YYLbd1fK2po= +github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= +github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= +github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= +github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= +github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= +github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= +github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= +github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= +github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 h1:ZpnhV/YsD2/4cESfV5+Hoeu/iUR3ruzNvZ+yQfO03a0= +github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2/go.mod h1:bBOAhwG1umN6/6ZUMtDFBMQR8jRg9O75tm9K00oMsK4= +github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= +github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/golang-jwt/jwt v3.2.1+incompatible h1:73Z+4BJcrTC+KczS6WvTPvRGOp1WmfEP4Q1lOd9Z/+c= +github.com/golang-jwt/jwt v3.2.1+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= +github.com/golang-jwt/jwt/v4 v4.4.3 h1:Hxl6lhQFj4AnOX6MLrsCb/+7tCj7DxP7VA+2rDIq5AU= +github.com/golang-jwt/jwt/v4 v4.4.3/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= +github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= +github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.1/go.mod h1:DopwsBzvsk0Fs44TXzsVbJyPhcCPeIwnvohx4u74HPM= +github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= +github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= +github.com/google/martian/v3 v3.1.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20201023163331-3e6fc7fc9c4c/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20210407192527-94a9f03dee38/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= +github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= +github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= +github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= +github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c h1:6rhixN/i8ZofjG1Y75iExal34USq5p+wiN1tpie8IrU= +github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c/go.mod h1:NMPJylDgVpX0MLRlPy15sqSwOFv/U1GZ2m21JhFfek0= +github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q= +github.com/hashicorp/consul/sdk v0.1.1/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= +github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= +github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= +github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= +github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= +github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= +github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= +github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= +github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= +github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= +github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= +github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= +github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= +github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/jawher/mow.cli v1.0.4/go.mod h1:5hQj2V8g+qYmLUVWqu4Wuja1pI57M83EChYLVZ0sMKk= +github.com/jawher/mow.cli v1.2.0/go.mod h1:y+pcA3jBAdo/GIZx/0rFjw/K2bVEODP9rfZOfaiq8Ko= +github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= +github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= +github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= +github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.14.4/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= +github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= +github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/linkedin/goavro/v2 v2.9.8/go.mod h1:UgQUb2N/pmueQYH9bfqFioWxzYCZXSfF8Jw03O5sjqA= +github.com/magiconair/properties v1.8.5/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= +github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= +github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= +github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= +github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= +github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= +github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= +github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= +github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/mitchellh/mapstructure v1.4.1/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/mtibben/percent v0.2.1 h1:5gssi8Nqo8QU/r2pynCm+hBQHpkB/uNK7BJCFogWdzs= +github.com/mtibben/percent v0.2.1/go.mod h1:KG9uO+SZkUp+VkRHsCdYQV3XSZrrSpR3O9ibNBTZrns= +github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= +github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= +github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= +github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= +github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= +github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= +github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= +github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= +github.com/onsi/ginkgo/v2 v2.1.3/go.mod h1:vw5CSIxN1JObi/U8gcbwft7ZxR2dgaR70JSE3/PpL4c= +github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= +github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= +github.com/onsi/gomega v1.17.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY= +github.com/onsi/gomega v1.19.0 h1:4ieX6qQjPP/BfC3mpsAtIGGlxTWPeA3Inl/7DtXw1tw= +github.com/onsi/gomega v1.19.0/go.mod h1:LY+I3pBVzYsTBU1AnDwOSxaYi9WoWiqgwooUqq9yPro= +github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= +github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= +github.com/pelletier/go-toml v1.9.3/go.mod h1:u1nR/EPcESfeI/szUZKdtJ0xRNbUoANCkoOuaOx1Y+c= +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= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= +github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= +github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= +github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= +github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= +github.com/prometheus/client_golang v1.11.1/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= +github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= +github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= +github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= +github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= +github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= +github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= +github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= +github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= +github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= +github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= +github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= +github.com/spf13/cast v1.3.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= +github.com/spf13/cobra v1.2.1/go.mod h1:ExllRjgxM/piMAM+3tAZvg8fsklGAf3tPfi+i8t68Nk= +github.com/spf13/jwalterweatherman v1.1.0/go.mod h1:aNWZUN0dPAAO/Ljvb5BEdw96iTZ0EXowPYD95IqWIGo= +github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= +github.com/spf13/viper v1.8.1/go.mod h1:o0Pch8wJ9BVSWGQMbra6iw0oQ5oktSIBaujf1rJH9Ns= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= +github.com/stretchr/objx v0.3.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= +github.com/stretchr/objx v0.4.0 h1:M2gUjqZET1qApGOWNSnZ49BAIMX4F/1plDv3+l31EJ4= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PKk= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= +github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +go.etcd.io/etcd/api/v3 v3.5.0/go.mod h1:cbVKeC6lCfl7j/8jBhAK6aIYO9XOjdptoxU/nLQcPvs= +go.etcd.io/etcd/client/pkg/v3 v3.5.0/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= +go.etcd.io/etcd/client/v2 v2.305.0/go.mod h1:h9puh54ZTgAKtEbut2oe9P4L/oqKCVB6xsXlzd7alYQ= +go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= +go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= +go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= +go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= +go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= +go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= +go.uber.org/zap v1.17.0/go.mod h1:MXVU+bhUf/A7Xi2HNOnopQOrmycQ5Ih87HtOu4q5SSo= +golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= +golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= +golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= +golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= +golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20201208152925-83fdc39ff7b5/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= +golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= +golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= +golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220225172249-27dd8689420f h1:oA4XRj0qtSt8Yo1Zms0CUlsT3KG69V2UGQWPBxujDmc= +golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210313182246-cd4f82c27b84/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 h1:0Ja1LBD+yisY6RWM/BH7TJVXWsSjs2VwBSmvSX4HdBc= +golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 h1:xHms4gcpe1YE7A3yIllJXP16CMAGuqwO2lX1mTyyRRc= +golang.org/x/sys v0.0.0-20220422013727-9388b58f7150/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 h1:JGgROgKl9N8DuW20oFS5gxc+lE67/N3FcwmBPMe7ArY= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191112195655-aa38f8e97acc/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200227222343-706bc42d1f0d/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= +golang.org/x/tools v0.0.0-20200312045724-11d5b4c81c7d/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= +golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200904185747-39188db58858/go.mod h1:Cj7w3i3Rnn0Xh82ur9kSqwfTHTeVxaDqrfMjpcNT6bE= +golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= +golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= +google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= +google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.19.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= +google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= +google.golang.org/api v0.35.0/go.mod h1:/XrVsuzM0rZmrsbjJutiuftIzeuTQcEeaYcSk/mQ1dg= +google.golang.org/api v0.36.0/go.mod h1:+z5ficQTmoYpPn8LCUNVpK5I7hwkpjbcgqA7I34qYtE= +google.golang.org/api v0.40.0/go.mod h1:fYKFpnQN0DsDSKRVRcQSDQNtqWPfM9i+zNPxepjRCQ8= +google.golang.org/api v0.41.0/go.mod h1:RkxM5lITDfTzmyKFPt+wGrCJbVfniCr2ool8kTBzRTU= +google.golang.org/api v0.43.0/go.mod h1:nQsDGjRXMo4lvh5hP0TKqF244gqhGcr/YSIykhUk/94= +google.golang.org/api v0.44.0/go.mod h1:EBOGZqzyhtvMDoxwS97ctnh0zUmYY6CxqXsc1AvkYD8= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= +google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6c= +google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= +google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= +google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= +google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= +google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= +google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200904004341-0bd0a958aa1d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20201109203340-2640f1f9cdfb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20201201144952-b05cb90ed32e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20201210142538-e3217bee35cc/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20201214200347-8c77b98c765d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210222152913-aa3ee6e6a81c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210303154014-9728d6b83eeb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210310155132-4ce2db91004e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210319143718-93e7006c17a6/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210402141018-6c239bbf2bb1/go.mod h1:9lPAdzaEmUacj36I+k7YKbEc5CXzPIeORRgDAUOu28A= +google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.31.1/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= +google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= +google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA51WJ8= +google.golang.org/grpc v1.35.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.36.1/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= +google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0 h1:bxAC2xTBsZGibn2RTntX0oH50xLsqy1OxA9tTL3p/lk= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLFVxaq6wH4YuVdsUOr75U= +gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= +gopkg.in/ini.v1 v1.62.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= +gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= +gopkg.in/square/go-jose.v2 v2.4.1/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= +gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= +rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= From f18cc6ec98c51c3ce2c22265e3ab69ab31f7492b Mon Sep 17 00:00:00 2001 From: Max Xu Date: Wed, 1 Mar 2023 15:31:54 +0800 Subject: [PATCH 238/348] chore: add gitignore Signed-off-by: Max Xu --- pulsaradmin/.gitignore | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 pulsaradmin/.gitignore diff --git a/pulsaradmin/.gitignore b/pulsaradmin/.gitignore new file mode 100644 index 0000000000..76a786dfd8 --- /dev/null +++ b/pulsaradmin/.gitignore @@ -0,0 +1,3 @@ +.DS_Store +.idea/ +.vscode/ From b353c8afd0f8eeb5c44a3a35619bd88a2c74d26a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 8 Mar 2023 21:17:33 +0800 Subject: [PATCH 239/348] Bump golang.org/x/net from 0.0.0-20220225172249-27dd8689420f to 0.7.0 (streamnative/pulsar-admin-go#2) Bumps [golang.org/x/net](/~https://github.com/golang/net) from 0.0.0-20220225172249-27dd8689420f to 0.7.0. - [Release notes](/~https://github.com/golang/net/releases) - [Commits](/~https://github.com/golang/net/commits/v0.7.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- pulsaradmin/go.mod | 6 +++--- pulsaradmin/go.sum | 8 +++++--- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index f1a1bbd0e2..b7b44c1a75 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -22,9 +22,9 @@ require ( github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/mtibben/percent v0.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - golang.org/x/net v0.0.0-20220225172249-27dd8689420f // indirect - golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 // indirect - golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 // indirect + golang.org/x/net v0.7.0 // indirect + golang.org/x/sys v0.5.0 // indirect + golang.org/x/term v0.5.0 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/protobuf v1.26.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/pulsaradmin/go.sum b/pulsaradmin/go.sum index 36562eafb7..75390df360 100644 --- a/pulsaradmin/go.sum +++ b/pulsaradmin/go.sum @@ -439,8 +439,9 @@ golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLd golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220225172249-27dd8689420f h1:oA4XRj0qtSt8Yo1Zms0CUlsT3KG69V2UGQWPBxujDmc= golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.7.0 h1:rJrUqqhjsgNp7KqAIc25s9pZnjU7TUcSY7HcVZjdn1g= +golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -468,8 +469,9 @@ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 h1:xHms4gcpe1YE7A3yIllJXP16CMAGuqwO2lX1mTyyRRc= golang.org/x/sys v0.0.0-20220422013727-9388b58f7150/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 h1:JGgROgKl9N8DuW20oFS5gxc+lE67/N3FcwmBPMe7ArY= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.5.0 h1:n2a8QNdAb0sZNpU9R1ALUXBbY+w51fCQDN+7EdxNBsY= +golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -478,8 +480,8 @@ golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.7.0 h1:4BRB4x83lYWy72KwLD/qYDuTu7q9PjSagHvijDw7cLo= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= From 8fa6c3a9cc3130842031846e357aef78d054e621 Mon Sep 17 00:00:00 2001 From: Max Xu Date: Thu, 9 Mar 2023 17:48:40 +0800 Subject: [PATCH 240/348] Optimize project layout (streamnative/pulsar-admin-go#3) * refactor: optimize project layout Signed-off-by: Max Xu --------- Signed-off-by: Max Xu --- pulsaradmin/pkg/{pulsar => admin}/admin.go | 33 +++++++++++-------- .../pkg/{pulsar => admin}/admin_test.go | 20 +++++------ .../{pulsar/common => admin}/api_version.go | 2 +- .../common => admin}/api_version_test.go | 2 +- .../pkg/{auth => admin}/auth_provider.go | 16 ++++----- .../pkg/{auth/tls.go => admin/auth_tls.go} | 2 +- .../{auth/token.go => admin/auth_token.go} | 10 +++--- .../pkg/{pulsar => admin}/broker_stats.go | 4 +-- pulsaradmin/pkg/{pulsar => admin}/brokers.go | 4 +-- pulsaradmin/pkg/{pulsar => admin}/cluster.go | 4 +-- .../pkg/{pulsar/common => admin}/config.go | 2 +- .../pkg/{pulsar => admin}/functions.go | 4 +-- .../pkg/{pulsar => admin}/functions_worker.go | 4 +-- .../pkg/{pulsar => admin}/namespace.go | 15 ++++----- .../{pulsar => admin}/ns_isolation_policy.go | 4 +-- pulsaradmin/pkg/{auth => admin}/oauth2.go | 10 +++--- .../pkg/{auth => admin}/oauth2_test.go | 2 +- pulsaradmin/pkg/{pulsar => admin}/packages.go | 4 +-- .../pkg/{pulsar => admin}/resource_quotas.go | 4 +-- pulsaradmin/pkg/{pulsar => admin}/schema.go | 4 +-- pulsaradmin/pkg/{pulsar => admin}/sinks.go | 4 +-- pulsaradmin/pkg/{pulsar => admin}/sources.go | 4 +-- .../pkg/{pulsar => admin}/subscription.go | 6 ++-- pulsaradmin/pkg/{pulsar => admin}/tenant.go | 4 +-- pulsaradmin/pkg/{pulsar => admin}/token.go | 6 ++-- .../pkg/{pulsar => admin}/token_test.go | 5 +-- pulsaradmin/pkg/{pulsar => admin}/topic.go | 15 ++++----- .../algorithm => }/algorithm/algorithm.go | 10 +++--- .../algorithm/algorithm_test.go | 0 .../common => }/algorithm/ecdsa/es256.go | 4 +-- .../common => }/algorithm/ecdsa/es384.go | 2 +- .../common => }/algorithm/ecdsa/es512.go | 4 +-- .../common => }/algorithm/hmac/hs256.go | 4 +-- .../common => }/algorithm/hmac/hs384.go | 4 +-- .../common => }/algorithm/hmac/hs512.go | 4 +-- .../common => }/algorithm/keypair/keypair.go | 0 .../common => }/algorithm/rsa/rs256.go | 4 +-- .../common => }/algorithm/rsa/rs384.go | 4 +-- .../common => }/algorithm/rsa/rs512.go | 4 +-- pulsaradmin/pkg/pulsar/defaults.go | 28 ---------------- pulsaradmin/pkg/{cli => rest}/client.go | 2 +- pulsaradmin/pkg/{cli => rest}/client_test.go | 2 +- pulsaradmin/pkg/{cli => rest}/errors.go | 2 +- .../pkg/{pulsar => }/utils/allocator_stats.go | 0 .../{pulsar/common => utils}/auth_action.go | 2 +- .../{pulsar/common => utils}/auth_polices.go | 2 +- .../common => utils}/auth_polices_test.go | 2 +- .../pkg/{pulsar => }/utils/backlog_quota.go | 0 .../utils/broker_ns_isolation_data.go | 0 .../pkg/{pulsar => }/utils/bundles_data.go | 0 .../utils/connector_definition.go | 0 .../pkg/{pulsar => }/utils/consumer_config.go | 0 pulsaradmin/pkg/{pulsar => }/utils/data.go | 0 .../pkg/{pulsar => }/utils/dispatch_rate.go | 0 .../pkg/{pulsar => }/utils/function_confg.go | 0 .../pkg/{pulsar => }/utils/function_state.go | 0 .../pkg/{pulsar => }/utils/function_status.go | 0 .../pkg/{pulsar => }/utils/functions_stats.go | 0 .../pkg/{pulsar => }/utils/home_dir.go | 0 .../utils/inactive_topic_policies.go | 0 .../utils/internal_configuration_data.go | 0 .../{pulsar => }/utils/load_manage_report.go | 0 .../utils/long_running_process_status.go | 0 pulsaradmin/pkg/{pulsar => }/utils/message.go | 0 .../pkg/{pulsar => }/utils/message_id.go | 0 .../pkg/{pulsar => }/utils/message_id_test.go | 0 pulsaradmin/pkg/{pulsar => }/utils/metrics.go | 0 .../pkg/{pulsar => }/utils/namespace_name.go | 0 .../{pulsar => }/utils/namespace_name_test.go | 0 .../{pulsar => }/utils/ns_isolation_data.go | 0 .../{pulsar => }/utils/ns_ownership_status.go | 0 .../{pulsar => }/utils/package_metadata.go | 0 .../pkg/{pulsar => }/utils/package_name.go | 0 .../{pulsar => }/utils/package_name_test.go | 0 .../pkg/{pulsar => }/utils/package_type.go | 0 .../utils/persistence_policies.go | 0 .../pkg/{pulsar => }/utils/policies.go | 8 ++--- .../pkg/{pulsar => }/utils/publish_rate.go | 0 .../pkg/{pulsar => }/utils/resource_quota.go | 0 .../pkg/{pulsar => }/utils/resources.go | 0 .../{pulsar => }/utils/retention_policies.go | 0 .../pkg/{pulsar => }/utils/schema_strategy.go | 0 .../pkg/{pulsar => }/utils/schema_util.go | 0 .../pkg/{pulsar => }/utils/sink_config.go | 0 .../pkg/{pulsar => }/utils/sink_status.go | 0 .../pkg/{pulsar => }/utils/source_config.go | 0 .../pkg/{pulsar => }/utils/source_status.go | 0 .../utils/subscription_auth_mode.go | 0 .../utils/topic_auto_creation_config.go | 0 .../pkg/{pulsar => }/utils/topic_domain.go | 0 .../pkg/{pulsar => }/utils/topic_name.go | 0 .../pkg/{pulsar => }/utils/topic_name_test.go | 0 .../pkg/{pulsar => }/utils/topic_type.go | 0 .../{pulsar => }/utils/topics_stats_stream.go | 0 .../pkg/{pulsar => }/utils/update_options.go | 0 pulsaradmin/pkg/{pulsar => }/utils/utils.go | 0 .../pkg/{pulsar => }/utils/utils_test.go | 0 .../pkg/{pulsar => }/utils/window_confing.go | 0 .../pkg/{pulsar => }/utils/worker_info.go | 0 99 files changed, 127 insertions(+), 159 deletions(-) rename pulsaradmin/pkg/{pulsar => admin}/admin.go (81%) rename pulsaradmin/pkg/{pulsar => admin}/admin_test.go (85%) rename pulsaradmin/pkg/{pulsar/common => admin}/api_version.go (98%) rename pulsaradmin/pkg/{pulsar/common => admin}/api_version_test.go (98%) rename pulsaradmin/pkg/{auth => admin}/auth_provider.go (88%) rename pulsaradmin/pkg/{auth/tls.go => admin/auth_tls.go} (99%) rename pulsaradmin/pkg/{auth/token.go => admin/auth_token.go} (92%) rename pulsaradmin/pkg/{pulsar => admin}/broker_stats.go (97%) rename pulsaradmin/pkg/{pulsar => admin}/brokers.go (98%) rename pulsaradmin/pkg/{pulsar => admin}/cluster.go (98%) rename pulsaradmin/pkg/{pulsar/common => admin}/config.go (99%) rename pulsaradmin/pkg/{pulsar => admin}/functions.go (99%) rename pulsaradmin/pkg/{pulsar => admin}/functions_worker.go (97%) rename pulsaradmin/pkg/{pulsar => admin}/namespace.go (99%) rename pulsaradmin/pkg/{pulsar => admin}/ns_isolation_policy.go (98%) rename pulsaradmin/pkg/{auth => admin}/oauth2.go (98%) rename pulsaradmin/pkg/{auth => admin}/oauth2_test.go (99%) rename pulsaradmin/pkg/{pulsar => admin}/packages.go (98%) rename pulsaradmin/pkg/{pulsar => admin}/resource_quotas.go (97%) rename pulsaradmin/pkg/{pulsar => admin}/schema.go (98%) rename pulsaradmin/pkg/{pulsar => admin}/sinks.go (99%) rename pulsaradmin/pkg/{pulsar => admin}/sources.go (99%) rename pulsaradmin/pkg/{pulsar => admin}/subscription.go (99%) rename pulsaradmin/pkg/{pulsar => admin}/tenant.go (96%) rename pulsaradmin/pkg/{pulsar => admin}/token.go (96%) rename pulsaradmin/pkg/{pulsar => admin}/token_test.go (94%) rename pulsaradmin/pkg/{pulsar => admin}/topic.go (98%) rename pulsaradmin/pkg/{pulsar/common/algorithm => }/algorithm/algorithm.go (87%) rename pulsaradmin/pkg/{pulsar/common/algorithm => }/algorithm/algorithm_test.go (100%) rename pulsaradmin/pkg/{pulsar/common => }/algorithm/ecdsa/es256.go (94%) rename pulsaradmin/pkg/{pulsar/common => }/algorithm/ecdsa/es384.go (94%) rename pulsaradmin/pkg/{pulsar/common => }/algorithm/ecdsa/es512.go (94%) rename pulsaradmin/pkg/{pulsar/common => }/algorithm/hmac/hs256.go (93%) rename pulsaradmin/pkg/{pulsar/common => }/algorithm/hmac/hs384.go (93%) rename pulsaradmin/pkg/{pulsar/common => }/algorithm/hmac/hs512.go (93%) rename pulsaradmin/pkg/{pulsar/common => }/algorithm/keypair/keypair.go (100%) rename pulsaradmin/pkg/{pulsar/common => }/algorithm/rsa/rs256.go (93%) rename pulsaradmin/pkg/{pulsar/common => }/algorithm/rsa/rs384.go (93%) rename pulsaradmin/pkg/{pulsar/common => }/algorithm/rsa/rs512.go (93%) delete mode 100644 pulsaradmin/pkg/pulsar/defaults.go rename pulsaradmin/pkg/{cli => rest}/client.go (99%) rename pulsaradmin/pkg/{cli => rest}/client_test.go (99%) rename pulsaradmin/pkg/{cli => rest}/errors.go (98%) rename pulsaradmin/pkg/{pulsar => }/utils/allocator_stats.go (100%) rename pulsaradmin/pkg/{pulsar/common => utils}/auth_action.go (99%) rename pulsaradmin/pkg/{pulsar/common => utils}/auth_polices.go (98%) rename pulsaradmin/pkg/{pulsar/common => utils}/auth_polices_test.go (99%) rename pulsaradmin/pkg/{pulsar => }/utils/backlog_quota.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/broker_ns_isolation_data.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/bundles_data.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/connector_definition.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/consumer_config.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/data.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/dispatch_rate.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/function_confg.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/function_state.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/function_status.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/functions_stats.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/home_dir.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/inactive_topic_policies.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/internal_configuration_data.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/load_manage_report.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/long_running_process_status.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/message.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/message_id.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/message_id_test.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/metrics.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/namespace_name.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/namespace_name_test.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/ns_isolation_data.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/ns_ownership_status.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/package_metadata.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/package_name.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/package_name_test.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/package_type.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/persistence_policies.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/policies.go (95%) rename pulsaradmin/pkg/{pulsar => }/utils/publish_rate.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/resource_quota.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/resources.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/retention_policies.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/schema_strategy.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/schema_util.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/sink_config.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/sink_status.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/source_config.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/source_status.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/subscription_auth_mode.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/topic_auto_creation_config.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/topic_domain.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/topic_name.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/topic_name_test.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/topic_type.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/topics_stats_stream.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/update_options.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/utils.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/utils_test.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/window_confing.go (100%) rename pulsaradmin/pkg/{pulsar => }/utils/worker_info.go (100%) diff --git a/pulsaradmin/pkg/pulsar/admin.go b/pulsaradmin/pkg/admin/admin.go similarity index 81% rename from pulsaradmin/pkg/pulsar/admin.go rename to pulsaradmin/pkg/admin/admin.go index 5f3eee3419..45cefbe092 100644 --- a/pulsaradmin/pkg/pulsar/admin.go +++ b/pulsaradmin/pkg/admin/admin.go @@ -15,17 +15,22 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "net/http" "net/url" "path" + "time" - "github.com/streamnative/pulsar-admin-go/pkg/auth" - "github.com/streamnative/pulsar-admin-go/pkg/cli" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/rest" + "github.com/streamnative/pulsar-admin-go/pkg/utils" +) + +const ( + DefaultWebServiceURL = "http://localhost:8080" + DefaultHTTPTimeOutDuration = 5 * time.Minute + ReleaseVersion = "None" ) type TLSOptions struct { @@ -54,13 +59,13 @@ type Client interface { } type pulsarClient struct { - Client *cli.Client - APIVersion common.APIVersion + Client *rest.Client + APIVersion APIVersion } // New returns a new client -func New(config *common.Config) (Client, error) { - authProvider, err := auth.GetAuthProvider(config) +func New(config *Config) (Client, error) { + authProvider, err := GetAuthProvider(config) if err != nil { return nil, err } @@ -69,7 +74,7 @@ func New(config *common.Config) (Client, error) { // NewWithAuthProvider creates a client with auth provider. // Deprecated: Use NewPulsarClientWithAuthProvider instead. -func NewWithAuthProvider(config *common.Config, authProvider auth.Provider) Client { +func NewWithAuthProvider(config *Config, authProvider AuthProvider) Client { client, err := NewPulsarClientWithAuthProvider(config, authProvider) if err != nil { panic(err) @@ -78,8 +83,8 @@ func NewWithAuthProvider(config *common.Config, authProvider auth.Provider) Clie } // NewPulsarClientWithAuthProvider create a client with auth provider. -func NewPulsarClientWithAuthProvider(config *common.Config, - authProvider auth.Provider) (Client, error) { +func NewPulsarClientWithAuthProvider(config *Config, + authProvider AuthProvider) (Client, error) { var transport http.RoundTripper if authProvider != nil { @@ -90,7 +95,7 @@ func NewPulsarClientWithAuthProvider(config *common.Config, } if transport == nil { - defaultTransport, err := auth.NewDefaultTransport(config) + defaultTransport, err := NewDefaultTransport(config) if err != nil { return nil, err } @@ -108,7 +113,7 @@ func NewPulsarClientWithAuthProvider(config *common.Config, c := &pulsarClient{ APIVersion: config.PulsarAPIVersion, - Client: &cli.Client{ + Client: &rest.Client{ ServiceURL: config.WebServiceURL, VersionInfo: ReleaseVersion, HTTPClient: &http.Client{ diff --git a/pulsaradmin/pkg/pulsar/admin_test.go b/pulsaradmin/pkg/admin/admin_test.go similarity index 85% rename from pulsaradmin/pkg/pulsar/admin_test.go rename to pulsaradmin/pkg/admin/admin_test.go index 4ab4fa413b..c3f9b521d5 100644 --- a/pulsaradmin/pkg/pulsar/admin_test.go +++ b/pulsaradmin/pkg/admin/admin_test.go @@ -15,36 +15,34 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "net/http" "testing" - "github.com/streamnative/pulsar-admin-go/pkg/auth" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) func TestPulsarClientEndpointEscapes(t *testing.T) { - client := pulsarClient{Client: nil, APIVersion: common.V2} + client := pulsarClient{Client: nil, APIVersion: V2} actual := client.endpoint("/myendpoint", "abc%? /def", "ghi") expected := "/admin/v2/myendpoint/abc%25%3F%20%2Fdef/ghi" assert.Equal(t, expected, actual) } func TestNew(t *testing.T) { - config := &common.Config{} + config := &Config{} admin, err := New(config) require.NoError(t, err) require.NotNil(t, admin) } func TestNewWithAuthProvider(t *testing.T) { - config := &common.Config{} + config := &Config{} - tokenAuth, err := auth.NewAuthenticationToken("eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9."+ + tokenAuth, err := NewAuthenticationToken("eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9."+ "eyJzdWIiOiJhZG1pbiIsImlhdCI6MTUxNjIzOTAyMn0.sVt6cyu3HKd89LcQvZVMNbqT0DTl3FvG9oYbj8hBDqU", nil) require.NoError(t, err) require.NotNil(t, tokenAuth) @@ -58,7 +56,7 @@ type customAuthProvider struct { transport http.RoundTripper } -var _ auth.Provider = &customAuthProvider{} +var _ AuthProvider = &customAuthProvider{} func (c *customAuthProvider) RoundTrip(req *http.Request) (*http.Response, error) { panic("implement me") @@ -73,8 +71,8 @@ func (c *customAuthProvider) WithTransport(transport http.RoundTripper) { } func TestNewWithCustomAuthProviderWithTransport(t *testing.T) { - config := &common.Config{} - defaultTransport, err := auth.NewDefaultTransport(config) + config := &Config{} + defaultTransport, err := NewDefaultTransport(config) require.NoError(t, err) customAuthProvider := &customAuthProvider{ @@ -90,7 +88,7 @@ func TestNewWithCustomAuthProviderWithTransport(t *testing.T) { } func TestNewWithTlsAllowInsecure(t *testing.T) { - config := &common.Config{ + config := &Config{ TLSAllowInsecureConnection: true, } admin, err := New(config) diff --git a/pulsaradmin/pkg/pulsar/common/api_version.go b/pulsaradmin/pkg/admin/api_version.go similarity index 98% rename from pulsaradmin/pkg/pulsar/common/api_version.go rename to pulsaradmin/pkg/admin/api_version.go index 6001c12f06..2347e864bf 100644 --- a/pulsaradmin/pkg/pulsar/common/api_version.go +++ b/pulsaradmin/pkg/admin/api_version.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package common +package admin type APIVersion int diff --git a/pulsaradmin/pkg/pulsar/common/api_version_test.go b/pulsaradmin/pkg/admin/api_version_test.go similarity index 98% rename from pulsaradmin/pkg/pulsar/common/api_version_test.go rename to pulsaradmin/pkg/admin/api_version_test.go index 335b774136..094fbd768b 100644 --- a/pulsaradmin/pkg/pulsar/common/api_version_test.go +++ b/pulsaradmin/pkg/admin/api_version_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package common +package admin import ( "testing" diff --git a/pulsaradmin/pkg/auth/auth_provider.go b/pulsaradmin/pkg/admin/auth_provider.go similarity index 88% rename from pulsaradmin/pkg/auth/auth_provider.go rename to pulsaradmin/pkg/admin/auth_provider.go index 3bcf8675b1..e04e7ea747 100644 --- a/pulsaradmin/pkg/auth/auth_provider.go +++ b/pulsaradmin/pkg/admin/auth_provider.go @@ -15,19 +15,17 @@ // specific language governing permissions and limitations // under the License. -package auth +package admin import ( "crypto/tls" "crypto/x509" "io/ioutil" "net/http" - - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" ) -// Provider provide a general method to add auth message -type Provider interface { +// AuthProvider provide a general method to add auth message +type AuthProvider interface { RoundTrip(req *http.Request) (*http.Response, error) Transport() http.RoundTripper WithTransport(tripper http.RoundTripper) @@ -37,8 +35,8 @@ type Transport struct { T http.RoundTripper } -func GetAuthProvider(config *common.Config) (Provider, error) { - var provider Provider +func GetAuthProvider(config *Config) (AuthProvider, error) { + var provider AuthProvider defaultTransport, err := NewDefaultTransport(config) if err != nil { return nil, err @@ -74,7 +72,7 @@ func GetAuthProvider(config *common.Config) (Provider, error) { // GetDefaultTransport gets a default transport. // Deprecated: Use NewDefaultTransport instead. -func GetDefaultTransport(config *common.Config) http.RoundTripper { +func GetDefaultTransport(config *Config) http.RoundTripper { transport, err := NewDefaultTransport(config) if err != nil { panic(err) @@ -83,7 +81,7 @@ func GetDefaultTransport(config *common.Config) http.RoundTripper { return transport } -func NewDefaultTransport(config *common.Config) (http.RoundTripper, error) { +func NewDefaultTransport(config *Config) (http.RoundTripper, error) { transport := http.DefaultTransport.(*http.Transport).Clone() tlsConfig := &tls.Config{ InsecureSkipVerify: config.TLSAllowInsecureConnection, diff --git a/pulsaradmin/pkg/auth/tls.go b/pulsaradmin/pkg/admin/auth_tls.go similarity index 99% rename from pulsaradmin/pkg/auth/tls.go rename to pulsaradmin/pkg/admin/auth_tls.go index 7c11fe27e7..d9473839f3 100644 --- a/pulsaradmin/pkg/auth/tls.go +++ b/pulsaradmin/pkg/admin/auth_tls.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package auth +package admin import ( "crypto/tls" diff --git a/pulsaradmin/pkg/auth/token.go b/pulsaradmin/pkg/admin/auth_token.go similarity index 92% rename from pulsaradmin/pkg/auth/token.go rename to pulsaradmin/pkg/admin/auth_token.go index 5b6eed6e8e..55fcd7170b 100644 --- a/pulsaradmin/pkg/auth/token.go +++ b/pulsaradmin/pkg/admin/auth_token.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package auth +package admin import ( "encoding/json" @@ -34,7 +34,7 @@ const ( TokePluginShortName = "token" ) -type Token struct { +type AuthToken struct { Token string `json:"token"` } @@ -43,7 +43,7 @@ type TokenAuthProvider struct { token string } -// NewAuthenticationToken return a interface of Provider with a string token. +// NewAuthenticationToken return a interface of AuthProvider with a string token. func NewAuthenticationToken(token string, transport http.RoundTripper) (*TokenAuthProvider, error) { if len(token) == 0 { return nil, errors.New("No token provided") @@ -51,7 +51,7 @@ func NewAuthenticationToken(token string, transport http.RoundTripper) (*TokenAu return &TokenAuthProvider{token: token, T: transport}, nil } -// NewAuthenticationTokenFromFile return a interface of a Provider with a string token file path. +// NewAuthenticationTokenFromFile return a interface of a AuthProvider with a string token file path. func NewAuthenticationTokenFromFile(tokenFilePath string, transport http.RoundTripper) (*TokenAuthProvider, error) { data, err := ioutil.ReadFile(tokenFilePath) if err != nil { @@ -66,7 +66,7 @@ func NewAuthenticationTokenFromAuthParams(encodedAuthParam string, var tokenAuthProvider *TokenAuthProvider var err error - var tokenJSON Token + var tokenJSON AuthToken err = json.Unmarshal([]byte(encodedAuthParam), &tokenJSON) if err != nil { switch { diff --git a/pulsaradmin/pkg/pulsar/broker_stats.go b/pulsaradmin/pkg/admin/broker_stats.go similarity index 97% rename from pulsaradmin/pkg/pulsar/broker_stats.go rename to pulsaradmin/pkg/admin/broker_stats.go index 387cae9d32..e235228d1f 100644 --- a/pulsaradmin/pkg/pulsar/broker_stats.go +++ b/pulsaradmin/pkg/admin/broker_stats.go @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // BrokerStats is admin interface for broker stats management diff --git a/pulsaradmin/pkg/pulsar/brokers.go b/pulsaradmin/pkg/admin/brokers.go similarity index 98% rename from pulsaradmin/pkg/pulsar/brokers.go rename to pulsaradmin/pkg/admin/brokers.go index da8d28c3fa..8a05433ad8 100644 --- a/pulsaradmin/pkg/pulsar/brokers.go +++ b/pulsaradmin/pkg/admin/brokers.go @@ -15,14 +15,14 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "fmt" "net/url" "strings" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // Brokers is admin interface for brokers management diff --git a/pulsaradmin/pkg/pulsar/cluster.go b/pulsaradmin/pkg/admin/cluster.go similarity index 98% rename from pulsaradmin/pkg/pulsar/cluster.go rename to pulsaradmin/pkg/admin/cluster.go index a08914323f..77bd5b5bd3 100644 --- a/pulsaradmin/pkg/pulsar/cluster.go +++ b/pulsaradmin/pkg/admin/cluster.go @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // Clusters is admin interface for clusters management diff --git a/pulsaradmin/pkg/pulsar/common/config.go b/pulsaradmin/pkg/admin/config.go similarity index 99% rename from pulsaradmin/pkg/pulsar/common/config.go rename to pulsaradmin/pkg/admin/config.go index 317e4bd34e..2639fbc139 100644 --- a/pulsaradmin/pkg/pulsar/common/config.go +++ b/pulsaradmin/pkg/admin/config.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package common +package admin type Config struct { // the web service url that pulsarctl connects to. Default is http://localhost:8080 diff --git a/pulsaradmin/pkg/pulsar/functions.go b/pulsaradmin/pkg/admin/functions.go similarity index 99% rename from pulsaradmin/pkg/pulsar/functions.go rename to pulsaradmin/pkg/admin/functions.go index 9af8216e7d..dd31d085bf 100644 --- a/pulsaradmin/pkg/pulsar/functions.go +++ b/pulsaradmin/pkg/admin/functions.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "bytes" @@ -28,7 +28,7 @@ import ( "path/filepath" "strings" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // Functions is admin interface for functions management diff --git a/pulsaradmin/pkg/pulsar/functions_worker.go b/pulsaradmin/pkg/admin/functions_worker.go similarity index 97% rename from pulsaradmin/pkg/pulsar/functions_worker.go rename to pulsaradmin/pkg/admin/functions_worker.go index 9703d157e8..fd285adffb 100644 --- a/pulsaradmin/pkg/pulsar/functions_worker.go +++ b/pulsaradmin/pkg/admin/functions_worker.go @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) type FunctionsWorker interface { diff --git a/pulsaradmin/pkg/pulsar/namespace.go b/pulsaradmin/pkg/admin/namespace.go similarity index 99% rename from pulsaradmin/pkg/pulsar/namespace.go rename to pulsaradmin/pkg/admin/namespace.go index 771ef6ac03..e8aaee3552 100644 --- a/pulsaradmin/pkg/pulsar/namespace.go +++ b/pulsaradmin/pkg/admin/namespace.go @@ -15,15 +15,14 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "net/url" "strconv" "strings" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // Namespaces is admin interface for namespaces management @@ -184,10 +183,10 @@ type Namespaces interface { SplitNamespaceBundle(namespace, bundle string, unloadSplitBundles bool) error // GetNamespacePermissions returns permissions on a namespace - GetNamespacePermissions(namespace utils.NameSpaceName) (map[string][]common.AuthAction, error) + GetNamespacePermissions(namespace utils.NameSpaceName) (map[string][]utils.AuthAction, error) // GrantNamespacePermission grants permission on a namespace. - GrantNamespacePermission(namespace utils.NameSpaceName, role string, action []common.AuthAction) error + GrantNamespacePermission(namespace utils.NameSpaceName, role string, action []utils.AuthAction) error // RevokeNamespacePermission revokes permissions on a namespace. RevokeNamespacePermission(namespace utils.NameSpaceName, role string) error @@ -702,15 +701,15 @@ func (n *namespaces) SplitNamespaceBundle(namespace, bundle string, unloadSplitB return n.pulsar.Client.PutWithQueryParams(endpoint, nil, nil, params) } -func (n *namespaces) GetNamespacePermissions(namespace utils.NameSpaceName) (map[string][]common.AuthAction, error) { +func (n *namespaces) GetNamespacePermissions(namespace utils.NameSpaceName) (map[string][]utils.AuthAction, error) { endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "permissions") - var permissions map[string][]common.AuthAction + var permissions map[string][]utils.AuthAction err := n.pulsar.Client.Get(endpoint, &permissions) return permissions, err } func (n *namespaces) GrantNamespacePermission(namespace utils.NameSpaceName, role string, - action []common.AuthAction) error { + action []utils.AuthAction) error { endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "permissions", role) s := make([]string, 0) for _, v := range action { diff --git a/pulsaradmin/pkg/pulsar/ns_isolation_policy.go b/pulsaradmin/pkg/admin/ns_isolation_policy.go similarity index 98% rename from pulsaradmin/pkg/pulsar/ns_isolation_policy.go rename to pulsaradmin/pkg/admin/ns_isolation_policy.go index 0ad8b03e18..a80c0ff353 100644 --- a/pulsaradmin/pkg/pulsar/ns_isolation_policy.go +++ b/pulsaradmin/pkg/admin/ns_isolation_policy.go @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) type NsIsolationPolicy interface { diff --git a/pulsaradmin/pkg/auth/oauth2.go b/pulsaradmin/pkg/admin/oauth2.go similarity index 98% rename from pulsaradmin/pkg/auth/oauth2.go rename to pulsaradmin/pkg/admin/oauth2.go index 09f8362703..c12d5d91f0 100644 --- a/pulsaradmin/pkg/auth/oauth2.go +++ b/pulsaradmin/pkg/admin/oauth2.go @@ -15,22 +15,22 @@ // specific language governing permissions and limitations // under the License. -package auth +package admin import ( "encoding/json" "net/http" "path/filepath" - "github.com/pkg/errors" - "github.com/99designs/keyring" "github.com/apache/pulsar-client-go/oauth2" "github.com/apache/pulsar-client-go/oauth2/cache" clock2 "github.com/apache/pulsar-client-go/oauth2/clock" "github.com/apache/pulsar-client-go/oauth2/store" - util "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/pkg/errors" xoauth2 "golang.org/x/oauth2" + + util "github.com/streamnative/pulsar-admin-go/pkg/utils" ) const ( @@ -71,7 +71,7 @@ func NewAuthenticationOAuth2(issuer oauth2.Issuer, store store.Store) (*OAuth2Pr } // NewAuthenticationOAuth2WithDefaultFlow uses memory to save the grant -func NewAuthenticationOAuth2WithDefaultFlow(issuer oauth2.Issuer, keyFile string) (Provider, error) { +func NewAuthenticationOAuth2WithDefaultFlow(issuer oauth2.Issuer, keyFile string) (AuthProvider, error) { st := store.NewMemoryStore() flow, err := oauth2.NewDefaultClientCredentialsFlow(oauth2.ClientCredentialsFlowOptions{ KeyFile: keyFile, diff --git a/pulsaradmin/pkg/auth/oauth2_test.go b/pulsaradmin/pkg/admin/oauth2_test.go similarity index 99% rename from pulsaradmin/pkg/auth/oauth2_test.go rename to pulsaradmin/pkg/admin/oauth2_test.go index f832af58cc..590f609f56 100644 --- a/pulsaradmin/pkg/auth/oauth2_test.go +++ b/pulsaradmin/pkg/admin/oauth2_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package auth +package admin import ( "fmt" diff --git a/pulsaradmin/pkg/pulsar/packages.go b/pulsaradmin/pkg/admin/packages.go similarity index 98% rename from pulsaradmin/pkg/pulsar/packages.go rename to pulsaradmin/pkg/admin/packages.go index ef9c773f97..2401c74658 100644 --- a/pulsaradmin/pkg/pulsar/packages.go +++ b/pulsaradmin/pkg/admin/packages.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "bytes" @@ -30,7 +30,7 @@ import ( "github.com/pkg/errors" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // Packages is admin interface for functions management diff --git a/pulsaradmin/pkg/pulsar/resource_quotas.go b/pulsaradmin/pkg/admin/resource_quotas.go similarity index 97% rename from pulsaradmin/pkg/pulsar/resource_quotas.go rename to pulsaradmin/pkg/admin/resource_quotas.go index 96e856f2f2..fd7c6d3ac9 100644 --- a/pulsaradmin/pkg/pulsar/resource_quotas.go +++ b/pulsaradmin/pkg/admin/resource_quotas.go @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) type ResourceQuotas interface { diff --git a/pulsaradmin/pkg/pulsar/schema.go b/pulsaradmin/pkg/admin/schema.go similarity index 98% rename from pulsaradmin/pkg/pulsar/schema.go rename to pulsaradmin/pkg/admin/schema.go index fb20dae34f..64b42f9834 100644 --- a/pulsaradmin/pkg/pulsar/schema.go +++ b/pulsaradmin/pkg/admin/schema.go @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "fmt" "strconv" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // Schema is admin interface for schema management diff --git a/pulsaradmin/pkg/pulsar/sinks.go b/pulsaradmin/pkg/admin/sinks.go similarity index 99% rename from pulsaradmin/pkg/pulsar/sinks.go rename to pulsaradmin/pkg/admin/sinks.go index 194b1d6ff2..26a66b32e6 100644 --- a/pulsaradmin/pkg/pulsar/sinks.go +++ b/pulsaradmin/pkg/admin/sinks.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "bytes" @@ -28,7 +28,7 @@ import ( "path/filepath" "strings" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // Sinks is admin interface for sinks management diff --git a/pulsaradmin/pkg/pulsar/sources.go b/pulsaradmin/pkg/admin/sources.go similarity index 99% rename from pulsaradmin/pkg/pulsar/sources.go rename to pulsaradmin/pkg/admin/sources.go index 1899e6e8d2..33b90640bc 100644 --- a/pulsaradmin/pkg/pulsar/sources.go +++ b/pulsaradmin/pkg/admin/sources.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "bytes" @@ -28,7 +28,7 @@ import ( "path/filepath" "strings" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // Sources is admin interface for sources management diff --git a/pulsaradmin/pkg/pulsar/subscription.go b/pulsaradmin/pkg/admin/subscription.go similarity index 99% rename from pulsaradmin/pkg/pulsar/subscription.go rename to pulsaradmin/pkg/admin/subscription.go index a17b381bd0..a3c240f3c3 100644 --- a/pulsaradmin/pkg/pulsar/subscription.go +++ b/pulsaradmin/pkg/admin/subscription.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "bytes" @@ -27,9 +27,9 @@ import ( "strconv" "strings" - // nolint "github.com/golang/protobuf/proto" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // Subscriptions is admin interface for subscriptions management diff --git a/pulsaradmin/pkg/pulsar/tenant.go b/pulsaradmin/pkg/admin/tenant.go similarity index 96% rename from pulsaradmin/pkg/pulsar/tenant.go rename to pulsaradmin/pkg/admin/tenant.go index 2d64540ff3..fba5b7a6d8 100644 --- a/pulsaradmin/pkg/pulsar/tenant.go +++ b/pulsaradmin/pkg/admin/tenant.go @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // Tenants is admin interface for tenants management diff --git a/pulsaradmin/pkg/pulsar/token.go b/pulsaradmin/pkg/admin/token.go similarity index 96% rename from pulsaradmin/pkg/pulsar/token.go rename to pulsaradmin/pkg/admin/token.go index c9263b94ec..7d2c20428b 100644 --- a/pulsaradmin/pkg/pulsar/token.go +++ b/pulsaradmin/pkg/admin/token.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "encoding/base64" @@ -24,8 +24,8 @@ import ( "github.com/golang-jwt/jwt/v4" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/algorithm" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + "github.com/streamnative/pulsar-admin-go/pkg/algorithm" + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" "github.com/pkg/errors" ) diff --git a/pulsaradmin/pkg/pulsar/token_test.go b/pulsaradmin/pkg/admin/token_test.go similarity index 94% rename from pulsaradmin/pkg/pulsar/token_test.go rename to pulsaradmin/pkg/admin/token_test.go index b30af6dab4..d8fe91c933 100644 --- a/pulsaradmin/pkg/pulsar/token_test.go +++ b/pulsaradmin/pkg/admin/token_test.go @@ -15,14 +15,15 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "testing" "time" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/algorithm" "github.com/stretchr/testify/require" + + "github.com/streamnative/pulsar-admin-go/pkg/algorithm" ) func TestCreateToken(t *testing.T) { diff --git a/pulsaradmin/pkg/pulsar/topic.go b/pulsaradmin/pkg/admin/topic.go similarity index 98% rename from pulsaradmin/pkg/pulsar/topic.go rename to pulsaradmin/pkg/admin/topic.go index e8cdcd9643..a1d7b2171e 100644 --- a/pulsaradmin/pkg/pulsar/topic.go +++ b/pulsaradmin/pkg/admin/topic.go @@ -15,14 +15,13 @@ // specific language governing permissions and limitations // under the License. -package pulsar +package admin import ( "fmt" "strconv" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/utils" + "github.com/streamnative/pulsar-admin-go/pkg/utils" ) // Topics is admin interface for topics management @@ -50,10 +49,10 @@ type Topics interface { // GetPermissions returns permissions on a topic // Retrieve the effective permissions for a topic. These permissions are defined by the permissions set at the // namespace level combined (union) with any eventual specific permission set on the topic. - GetPermissions(utils.TopicName) (map[string][]common.AuthAction, error) + GetPermissions(utils.TopicName) (map[string][]utils.AuthAction, error) // GrantPermission grants a new permission to a client role on a single topic - GrantPermission(utils.TopicName, string, []common.AuthAction) error + GrantPermission(utils.TopicName, string, []utils.AuthAction) error // RevokePermission revokes permissions to a client role on a single topic. If the permission // was not set at the topic level, but rather at the namespace level, this operation will @@ -339,14 +338,14 @@ func (t *topics) GetInternalInfo(topic utils.TopicName) (utils.ManagedLedgerInfo return info, err } -func (t *topics) GetPermissions(topic utils.TopicName) (map[string][]common.AuthAction, error) { - var permissions map[string][]common.AuthAction +func (t *topics) GetPermissions(topic utils.TopicName) (map[string][]utils.AuthAction, error) { + var permissions map[string][]utils.AuthAction endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "permissions") err := t.pulsar.Client.Get(endpoint, &permissions) return permissions, err } -func (t *topics) GrantPermission(topic utils.TopicName, role string, action []common.AuthAction) error { +func (t *topics) GrantPermission(topic utils.TopicName, role string, action []utils.AuthAction) error { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "permissions", role) s := []string{} for _, v := range action { diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go b/pulsaradmin/pkg/algorithm/algorithm.go similarity index 87% rename from pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go rename to pulsaradmin/pkg/algorithm/algorithm.go index 75ff5dac19..bd66da531e 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm.go +++ b/pulsaradmin/pkg/algorithm/algorithm.go @@ -18,12 +18,12 @@ package algorithm import ( - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/ecdsa" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/hmac" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/rsa" - "github.com/pkg/errors" + + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/ecdsa" + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/hmac" + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/rsa" ) type Algorithm string diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go b/pulsaradmin/pkg/algorithm/algorithm_test.go similarity index 100% rename from pulsaradmin/pkg/pulsar/common/algorithm/algorithm/algorithm_test.go rename to pulsaradmin/pkg/algorithm/algorithm_test.go diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es256.go b/pulsaradmin/pkg/algorithm/ecdsa/es256.go similarity index 94% rename from pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es256.go rename to pulsaradmin/pkg/algorithm/ecdsa/es256.go index 0e652e6e01..f6334554bc 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es256.go +++ b/pulsaradmin/pkg/algorithm/ecdsa/es256.go @@ -22,9 +22,9 @@ import ( "crypto/elliptic" "crypto/rand" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" - "github.com/pkg/errors" + + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" ) type ES256 struct{} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go b/pulsaradmin/pkg/algorithm/ecdsa/es384.go similarity index 94% rename from pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go rename to pulsaradmin/pkg/algorithm/ecdsa/es384.go index b2fdd20444..fe6d3c48eb 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es384.go +++ b/pulsaradmin/pkg/algorithm/ecdsa/es384.go @@ -22,7 +22,7 @@ import ( "crypto/elliptic" "crypto/rand" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" "github.com/pkg/errors" ) diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go b/pulsaradmin/pkg/algorithm/ecdsa/es512.go similarity index 94% rename from pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go rename to pulsaradmin/pkg/algorithm/ecdsa/es512.go index bc50349f06..64d7498d0e 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/ecdsa/es512.go +++ b/pulsaradmin/pkg/algorithm/ecdsa/es512.go @@ -22,9 +22,9 @@ import ( "crypto/elliptic" "crypto/rand" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" - "github.com/pkg/errors" + + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" ) type ES512 struct{} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs256.go b/pulsaradmin/pkg/algorithm/hmac/hs256.go similarity index 93% rename from pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs256.go rename to pulsaradmin/pkg/algorithm/hmac/hs256.go index b2990cca67..341df6b00f 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs256.go +++ b/pulsaradmin/pkg/algorithm/hmac/hs256.go @@ -22,9 +22,9 @@ import ( "crypto/rand" "crypto/sha256" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" - "github.com/pkg/errors" + + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" ) type HS256 struct{} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs384.go b/pulsaradmin/pkg/algorithm/hmac/hs384.go similarity index 93% rename from pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs384.go rename to pulsaradmin/pkg/algorithm/hmac/hs384.go index ad226be962..4ff2ad2e75 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs384.go +++ b/pulsaradmin/pkg/algorithm/hmac/hs384.go @@ -22,9 +22,9 @@ import ( "crypto/rand" "crypto/sha512" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" - "github.com/pkg/errors" + + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" ) type HS384 struct{} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs512.go b/pulsaradmin/pkg/algorithm/hmac/hs512.go similarity index 93% rename from pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs512.go rename to pulsaradmin/pkg/algorithm/hmac/hs512.go index ed61310519..761d43e60c 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/hmac/hs512.go +++ b/pulsaradmin/pkg/algorithm/hmac/hs512.go @@ -22,9 +22,9 @@ import ( "crypto/rand" "crypto/sha512" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" - "github.com/pkg/errors" + + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" ) type HS512 struct{} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go b/pulsaradmin/pkg/algorithm/keypair/keypair.go similarity index 100% rename from pulsaradmin/pkg/pulsar/common/algorithm/keypair/keypair.go rename to pulsaradmin/pkg/algorithm/keypair/keypair.go diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs256.go b/pulsaradmin/pkg/algorithm/rsa/rs256.go similarity index 93% rename from pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs256.go rename to pulsaradmin/pkg/algorithm/rsa/rs256.go index e104039e38..11fb320f31 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs256.go +++ b/pulsaradmin/pkg/algorithm/rsa/rs256.go @@ -21,9 +21,9 @@ import ( "crypto/rand" "crypto/rsa" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" - "github.com/pkg/errors" + + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" ) type RS256 struct{} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs384.go b/pulsaradmin/pkg/algorithm/rsa/rs384.go similarity index 93% rename from pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs384.go rename to pulsaradmin/pkg/algorithm/rsa/rs384.go index a4a8280308..ea6d984672 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs384.go +++ b/pulsaradmin/pkg/algorithm/rsa/rs384.go @@ -21,9 +21,9 @@ import ( "crypto/rand" "crypto/rsa" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" - "github.com/pkg/errors" + + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" ) type RS384 struct{} diff --git a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs512.go b/pulsaradmin/pkg/algorithm/rsa/rs512.go similarity index 93% rename from pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs512.go rename to pulsaradmin/pkg/algorithm/rsa/rs512.go index 5aa5450c74..fbe7da0d2b 100644 --- a/pulsaradmin/pkg/pulsar/common/algorithm/rsa/rs512.go +++ b/pulsaradmin/pkg/algorithm/rsa/rs512.go @@ -21,9 +21,9 @@ import ( "crypto/rand" "crypto/rsa" - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common/algorithm/keypair" - "github.com/pkg/errors" + + "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" ) type RS512 struct{} diff --git a/pulsaradmin/pkg/pulsar/defaults.go b/pulsaradmin/pkg/pulsar/defaults.go deleted file mode 100644 index 50e67bcb32..0000000000 --- a/pulsaradmin/pkg/pulsar/defaults.go +++ /dev/null @@ -1,28 +0,0 @@ -// 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 pulsar - -import ( - "time" -) - -const ( - DefaultWebServiceURL = "http://localhost:8080" - DefaultHTTPTimeOutDuration = 5 * time.Minute - ReleaseVersion = "None" -) diff --git a/pulsaradmin/pkg/cli/client.go b/pulsaradmin/pkg/rest/client.go similarity index 99% rename from pulsaradmin/pkg/cli/client.go rename to pulsaradmin/pkg/rest/client.go index 42f9147667..10ac5473bc 100644 --- a/pulsaradmin/pkg/cli/client.go +++ b/pulsaradmin/pkg/rest/client.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package cli +package rest import ( "bytes" diff --git a/pulsaradmin/pkg/cli/client_test.go b/pulsaradmin/pkg/rest/client_test.go similarity index 99% rename from pulsaradmin/pkg/cli/client_test.go rename to pulsaradmin/pkg/rest/client_test.go index f5335946b7..f82804fca1 100644 --- a/pulsaradmin/pkg/cli/client_test.go +++ b/pulsaradmin/pkg/rest/client_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package cli +package rest import ( "io/ioutil" diff --git a/pulsaradmin/pkg/cli/errors.go b/pulsaradmin/pkg/rest/errors.go similarity index 98% rename from pulsaradmin/pkg/cli/errors.go rename to pulsaradmin/pkg/rest/errors.go index e7e2ecbad5..dc611ec388 100644 --- a/pulsaradmin/pkg/cli/errors.go +++ b/pulsaradmin/pkg/rest/errors.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package cli +package rest import "fmt" diff --git a/pulsaradmin/pkg/pulsar/utils/allocator_stats.go b/pulsaradmin/pkg/utils/allocator_stats.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/allocator_stats.go rename to pulsaradmin/pkg/utils/allocator_stats.go diff --git a/pulsaradmin/pkg/pulsar/common/auth_action.go b/pulsaradmin/pkg/utils/auth_action.go similarity index 99% rename from pulsaradmin/pkg/pulsar/common/auth_action.go rename to pulsaradmin/pkg/utils/auth_action.go index 0a116a673e..7f2bf25735 100644 --- a/pulsaradmin/pkg/pulsar/common/auth_action.go +++ b/pulsaradmin/pkg/utils/auth_action.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package common +package utils import "github.com/pkg/errors" diff --git a/pulsaradmin/pkg/pulsar/common/auth_polices.go b/pulsaradmin/pkg/utils/auth_polices.go similarity index 98% rename from pulsaradmin/pkg/pulsar/common/auth_polices.go rename to pulsaradmin/pkg/utils/auth_polices.go index ed97b3fbd6..065b3c434d 100644 --- a/pulsaradmin/pkg/pulsar/common/auth_polices.go +++ b/pulsaradmin/pkg/utils/auth_polices.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package common +package utils type AuthPolicies struct { NamespaceAuth map[string][]AuthAction `json:"namespace_auth"` diff --git a/pulsaradmin/pkg/pulsar/common/auth_polices_test.go b/pulsaradmin/pkg/utils/auth_polices_test.go similarity index 99% rename from pulsaradmin/pkg/pulsar/common/auth_polices_test.go rename to pulsaradmin/pkg/utils/auth_polices_test.go index 37a39ce05c..014f594bdc 100644 --- a/pulsaradmin/pkg/pulsar/common/auth_polices_test.go +++ b/pulsaradmin/pkg/utils/auth_polices_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package common +package utils import ( "encoding/json" diff --git a/pulsaradmin/pkg/pulsar/utils/backlog_quota.go b/pulsaradmin/pkg/utils/backlog_quota.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/backlog_quota.go rename to pulsaradmin/pkg/utils/backlog_quota.go diff --git a/pulsaradmin/pkg/pulsar/utils/broker_ns_isolation_data.go b/pulsaradmin/pkg/utils/broker_ns_isolation_data.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/broker_ns_isolation_data.go rename to pulsaradmin/pkg/utils/broker_ns_isolation_data.go diff --git a/pulsaradmin/pkg/pulsar/utils/bundles_data.go b/pulsaradmin/pkg/utils/bundles_data.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/bundles_data.go rename to pulsaradmin/pkg/utils/bundles_data.go diff --git a/pulsaradmin/pkg/pulsar/utils/connector_definition.go b/pulsaradmin/pkg/utils/connector_definition.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/connector_definition.go rename to pulsaradmin/pkg/utils/connector_definition.go diff --git a/pulsaradmin/pkg/pulsar/utils/consumer_config.go b/pulsaradmin/pkg/utils/consumer_config.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/consumer_config.go rename to pulsaradmin/pkg/utils/consumer_config.go diff --git a/pulsaradmin/pkg/pulsar/utils/data.go b/pulsaradmin/pkg/utils/data.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/data.go rename to pulsaradmin/pkg/utils/data.go diff --git a/pulsaradmin/pkg/pulsar/utils/dispatch_rate.go b/pulsaradmin/pkg/utils/dispatch_rate.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/dispatch_rate.go rename to pulsaradmin/pkg/utils/dispatch_rate.go diff --git a/pulsaradmin/pkg/pulsar/utils/function_confg.go b/pulsaradmin/pkg/utils/function_confg.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/function_confg.go rename to pulsaradmin/pkg/utils/function_confg.go diff --git a/pulsaradmin/pkg/pulsar/utils/function_state.go b/pulsaradmin/pkg/utils/function_state.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/function_state.go rename to pulsaradmin/pkg/utils/function_state.go diff --git a/pulsaradmin/pkg/pulsar/utils/function_status.go b/pulsaradmin/pkg/utils/function_status.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/function_status.go rename to pulsaradmin/pkg/utils/function_status.go diff --git a/pulsaradmin/pkg/pulsar/utils/functions_stats.go b/pulsaradmin/pkg/utils/functions_stats.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/functions_stats.go rename to pulsaradmin/pkg/utils/functions_stats.go diff --git a/pulsaradmin/pkg/pulsar/utils/home_dir.go b/pulsaradmin/pkg/utils/home_dir.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/home_dir.go rename to pulsaradmin/pkg/utils/home_dir.go diff --git a/pulsaradmin/pkg/pulsar/utils/inactive_topic_policies.go b/pulsaradmin/pkg/utils/inactive_topic_policies.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/inactive_topic_policies.go rename to pulsaradmin/pkg/utils/inactive_topic_policies.go diff --git a/pulsaradmin/pkg/pulsar/utils/internal_configuration_data.go b/pulsaradmin/pkg/utils/internal_configuration_data.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/internal_configuration_data.go rename to pulsaradmin/pkg/utils/internal_configuration_data.go diff --git a/pulsaradmin/pkg/pulsar/utils/load_manage_report.go b/pulsaradmin/pkg/utils/load_manage_report.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/load_manage_report.go rename to pulsaradmin/pkg/utils/load_manage_report.go diff --git a/pulsaradmin/pkg/pulsar/utils/long_running_process_status.go b/pulsaradmin/pkg/utils/long_running_process_status.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/long_running_process_status.go rename to pulsaradmin/pkg/utils/long_running_process_status.go diff --git a/pulsaradmin/pkg/pulsar/utils/message.go b/pulsaradmin/pkg/utils/message.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/message.go rename to pulsaradmin/pkg/utils/message.go diff --git a/pulsaradmin/pkg/pulsar/utils/message_id.go b/pulsaradmin/pkg/utils/message_id.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/message_id.go rename to pulsaradmin/pkg/utils/message_id.go diff --git a/pulsaradmin/pkg/pulsar/utils/message_id_test.go b/pulsaradmin/pkg/utils/message_id_test.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/message_id_test.go rename to pulsaradmin/pkg/utils/message_id_test.go diff --git a/pulsaradmin/pkg/pulsar/utils/metrics.go b/pulsaradmin/pkg/utils/metrics.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/metrics.go rename to pulsaradmin/pkg/utils/metrics.go diff --git a/pulsaradmin/pkg/pulsar/utils/namespace_name.go b/pulsaradmin/pkg/utils/namespace_name.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/namespace_name.go rename to pulsaradmin/pkg/utils/namespace_name.go diff --git a/pulsaradmin/pkg/pulsar/utils/namespace_name_test.go b/pulsaradmin/pkg/utils/namespace_name_test.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/namespace_name_test.go rename to pulsaradmin/pkg/utils/namespace_name_test.go diff --git a/pulsaradmin/pkg/pulsar/utils/ns_isolation_data.go b/pulsaradmin/pkg/utils/ns_isolation_data.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/ns_isolation_data.go rename to pulsaradmin/pkg/utils/ns_isolation_data.go diff --git a/pulsaradmin/pkg/pulsar/utils/ns_ownership_status.go b/pulsaradmin/pkg/utils/ns_ownership_status.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/ns_ownership_status.go rename to pulsaradmin/pkg/utils/ns_ownership_status.go diff --git a/pulsaradmin/pkg/pulsar/utils/package_metadata.go b/pulsaradmin/pkg/utils/package_metadata.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/package_metadata.go rename to pulsaradmin/pkg/utils/package_metadata.go diff --git a/pulsaradmin/pkg/pulsar/utils/package_name.go b/pulsaradmin/pkg/utils/package_name.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/package_name.go rename to pulsaradmin/pkg/utils/package_name.go diff --git a/pulsaradmin/pkg/pulsar/utils/package_name_test.go b/pulsaradmin/pkg/utils/package_name_test.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/package_name_test.go rename to pulsaradmin/pkg/utils/package_name_test.go diff --git a/pulsaradmin/pkg/pulsar/utils/package_type.go b/pulsaradmin/pkg/utils/package_type.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/package_type.go rename to pulsaradmin/pkg/utils/package_type.go diff --git a/pulsaradmin/pkg/pulsar/utils/persistence_policies.go b/pulsaradmin/pkg/utils/persistence_policies.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/persistence_policies.go rename to pulsaradmin/pkg/utils/persistence_policies.go diff --git a/pulsaradmin/pkg/pulsar/utils/policies.go b/pulsaradmin/pkg/utils/policies.go similarity index 95% rename from pulsaradmin/pkg/pulsar/utils/policies.go rename to pulsaradmin/pkg/utils/policies.go index 9822bb0c65..3d727994c3 100644 --- a/pulsaradmin/pkg/pulsar/utils/policies.go +++ b/pulsaradmin/pkg/utils/policies.go @@ -17,10 +17,6 @@ package utils -import ( - "github.com/streamnative/pulsar-admin-go/pkg/pulsar/common" -) - const ( FirstBoundary string = "0x00000000" LastBoundary string = "0xffffffff" @@ -52,14 +48,14 @@ type Policies struct { ClusterSubscribeRate map[string]SubscribeRate `json:"clusterSubscribeRate"` TopicAutoCreationConfig *TopicAutoCreationConfig `json:"autoTopicCreationOverride"` SchemaCompatibilityStrategy SchemaCompatibilityStrategy `json:"schema_auto_update_compatibility_strategy"` - AuthPolicies common.AuthPolicies `json:"auth_policies"` + AuthPolicies AuthPolicies `json:"auth_policies"` SubscriptionAuthMode SubscriptionAuthMode `json:"subscription_auth_mode"` IsAllowAutoUpdateSchema *bool `json:"is_allow_auto_update_schema"` } func NewDefaultPolicies() *Policies { return &Policies{ - AuthPolicies: *common.NewAuthPolicies(), + AuthPolicies: *NewAuthPolicies(), ReplicationClusters: make([]string, 0, 10), BacklogQuotaMap: make(map[BacklogQuotaType]BacklogQuota), TopicDispatchRate: make(map[string]DispatchRate), diff --git a/pulsaradmin/pkg/pulsar/utils/publish_rate.go b/pulsaradmin/pkg/utils/publish_rate.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/publish_rate.go rename to pulsaradmin/pkg/utils/publish_rate.go diff --git a/pulsaradmin/pkg/pulsar/utils/resource_quota.go b/pulsaradmin/pkg/utils/resource_quota.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/resource_quota.go rename to pulsaradmin/pkg/utils/resource_quota.go diff --git a/pulsaradmin/pkg/pulsar/utils/resources.go b/pulsaradmin/pkg/utils/resources.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/resources.go rename to pulsaradmin/pkg/utils/resources.go diff --git a/pulsaradmin/pkg/pulsar/utils/retention_policies.go b/pulsaradmin/pkg/utils/retention_policies.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/retention_policies.go rename to pulsaradmin/pkg/utils/retention_policies.go diff --git a/pulsaradmin/pkg/pulsar/utils/schema_strategy.go b/pulsaradmin/pkg/utils/schema_strategy.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/schema_strategy.go rename to pulsaradmin/pkg/utils/schema_strategy.go diff --git a/pulsaradmin/pkg/pulsar/utils/schema_util.go b/pulsaradmin/pkg/utils/schema_util.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/schema_util.go rename to pulsaradmin/pkg/utils/schema_util.go diff --git a/pulsaradmin/pkg/pulsar/utils/sink_config.go b/pulsaradmin/pkg/utils/sink_config.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/sink_config.go rename to pulsaradmin/pkg/utils/sink_config.go diff --git a/pulsaradmin/pkg/pulsar/utils/sink_status.go b/pulsaradmin/pkg/utils/sink_status.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/sink_status.go rename to pulsaradmin/pkg/utils/sink_status.go diff --git a/pulsaradmin/pkg/pulsar/utils/source_config.go b/pulsaradmin/pkg/utils/source_config.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/source_config.go rename to pulsaradmin/pkg/utils/source_config.go diff --git a/pulsaradmin/pkg/pulsar/utils/source_status.go b/pulsaradmin/pkg/utils/source_status.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/source_status.go rename to pulsaradmin/pkg/utils/source_status.go diff --git a/pulsaradmin/pkg/pulsar/utils/subscription_auth_mode.go b/pulsaradmin/pkg/utils/subscription_auth_mode.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/subscription_auth_mode.go rename to pulsaradmin/pkg/utils/subscription_auth_mode.go diff --git a/pulsaradmin/pkg/pulsar/utils/topic_auto_creation_config.go b/pulsaradmin/pkg/utils/topic_auto_creation_config.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/topic_auto_creation_config.go rename to pulsaradmin/pkg/utils/topic_auto_creation_config.go diff --git a/pulsaradmin/pkg/pulsar/utils/topic_domain.go b/pulsaradmin/pkg/utils/topic_domain.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/topic_domain.go rename to pulsaradmin/pkg/utils/topic_domain.go diff --git a/pulsaradmin/pkg/pulsar/utils/topic_name.go b/pulsaradmin/pkg/utils/topic_name.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/topic_name.go rename to pulsaradmin/pkg/utils/topic_name.go diff --git a/pulsaradmin/pkg/pulsar/utils/topic_name_test.go b/pulsaradmin/pkg/utils/topic_name_test.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/topic_name_test.go rename to pulsaradmin/pkg/utils/topic_name_test.go diff --git a/pulsaradmin/pkg/pulsar/utils/topic_type.go b/pulsaradmin/pkg/utils/topic_type.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/topic_type.go rename to pulsaradmin/pkg/utils/topic_type.go diff --git a/pulsaradmin/pkg/pulsar/utils/topics_stats_stream.go b/pulsaradmin/pkg/utils/topics_stats_stream.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/topics_stats_stream.go rename to pulsaradmin/pkg/utils/topics_stats_stream.go diff --git a/pulsaradmin/pkg/pulsar/utils/update_options.go b/pulsaradmin/pkg/utils/update_options.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/update_options.go rename to pulsaradmin/pkg/utils/update_options.go diff --git a/pulsaradmin/pkg/pulsar/utils/utils.go b/pulsaradmin/pkg/utils/utils.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/utils.go rename to pulsaradmin/pkg/utils/utils.go diff --git a/pulsaradmin/pkg/pulsar/utils/utils_test.go b/pulsaradmin/pkg/utils/utils_test.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/utils_test.go rename to pulsaradmin/pkg/utils/utils_test.go diff --git a/pulsaradmin/pkg/pulsar/utils/window_confing.go b/pulsaradmin/pkg/utils/window_confing.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/window_confing.go rename to pulsaradmin/pkg/utils/window_confing.go diff --git a/pulsaradmin/pkg/pulsar/utils/worker_info.go b/pulsaradmin/pkg/utils/worker_info.go similarity index 100% rename from pulsaradmin/pkg/pulsar/utils/worker_info.go rename to pulsaradmin/pkg/utils/worker_info.go From 786076acb6a3749aff2f5235efceba3ed7e220cd Mon Sep 17 00:00:00 2001 From: Max Xu Date: Tue, 14 Mar 2023 10:32:00 +0800 Subject: [PATCH 241/348] Decouple config and auth package (streamnative/pulsar-admin-go#4) chore: decouple config and auth package Signed-off-by: Max Xu --- pulsaradmin/pkg/admin/admin.go | 16 ++--- pulsaradmin/pkg/admin/admin_test.go | 19 +++--- pulsaradmin/pkg/admin/{ => auth}/oauth2.go | 4 +- .../pkg/admin/{ => auth}/oauth2_test.go | 2 +- .../{auth_provider.go => auth/provider.go} | 48 +++------------ .../pkg/admin/{auth_tls.go => auth/tls.go} | 2 +- .../admin/{auth_token.go => auth/token.go} | 10 ++-- pulsaradmin/pkg/admin/auth/transport.go | 60 +++++++++++++++++++ .../pkg/admin/{ => config}/api_version.go | 2 +- .../admin/{ => config}/api_version_test.go | 2 +- pulsaradmin/pkg/admin/{ => config}/config.go | 2 +- 11 files changed, 99 insertions(+), 68 deletions(-) rename pulsaradmin/pkg/admin/{ => auth}/oauth2.go (99%) rename pulsaradmin/pkg/admin/{ => auth}/oauth2_test.go (99%) rename pulsaradmin/pkg/admin/{auth_provider.go => auth/provider.go} (67%) rename pulsaradmin/pkg/admin/{auth_tls.go => auth/tls.go} (99%) rename pulsaradmin/pkg/admin/{auth_token.go => auth/token.go} (92%) create mode 100644 pulsaradmin/pkg/admin/auth/transport.go rename pulsaradmin/pkg/admin/{ => config}/api_version.go (98%) rename pulsaradmin/pkg/admin/{ => config}/api_version_test.go (98%) rename pulsaradmin/pkg/admin/{ => config}/config.go (99%) diff --git a/pulsaradmin/pkg/admin/admin.go b/pulsaradmin/pkg/admin/admin.go index 45cefbe092..8931d7dd81 100644 --- a/pulsaradmin/pkg/admin/admin.go +++ b/pulsaradmin/pkg/admin/admin.go @@ -23,6 +23,8 @@ import ( "path" "time" + "github.com/streamnative/pulsar-admin-go/pkg/admin/auth" + "github.com/streamnative/pulsar-admin-go/pkg/admin/config" "github.com/streamnative/pulsar-admin-go/pkg/rest" "github.com/streamnative/pulsar-admin-go/pkg/utils" ) @@ -60,12 +62,12 @@ type Client interface { type pulsarClient struct { Client *rest.Client - APIVersion APIVersion + APIVersion config.APIVersion } // New returns a new client -func New(config *Config) (Client, error) { - authProvider, err := GetAuthProvider(config) +func New(config *config.Config) (Client, error) { + authProvider, err := auth.GetAuthProvider(config) if err != nil { return nil, err } @@ -74,7 +76,7 @@ func New(config *Config) (Client, error) { // NewWithAuthProvider creates a client with auth provider. // Deprecated: Use NewPulsarClientWithAuthProvider instead. -func NewWithAuthProvider(config *Config, authProvider AuthProvider) Client { +func NewWithAuthProvider(config *config.Config, authProvider auth.Provider) Client { client, err := NewPulsarClientWithAuthProvider(config, authProvider) if err != nil { panic(err) @@ -83,8 +85,8 @@ func NewWithAuthProvider(config *Config, authProvider AuthProvider) Client { } // NewPulsarClientWithAuthProvider create a client with auth provider. -func NewPulsarClientWithAuthProvider(config *Config, - authProvider AuthProvider) (Client, error) { +func NewPulsarClientWithAuthProvider(config *config.Config, + authProvider auth.Provider) (Client, error) { var transport http.RoundTripper if authProvider != nil { @@ -95,7 +97,7 @@ func NewPulsarClientWithAuthProvider(config *Config, } if transport == nil { - defaultTransport, err := NewDefaultTransport(config) + defaultTransport, err := auth.NewDefaultTransport(config) if err != nil { return nil, err } diff --git a/pulsaradmin/pkg/admin/admin_test.go b/pulsaradmin/pkg/admin/admin_test.go index c3f9b521d5..9a51fbaec7 100644 --- a/pulsaradmin/pkg/admin/admin_test.go +++ b/pulsaradmin/pkg/admin/admin_test.go @@ -23,26 +23,29 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/streamnative/pulsar-admin-go/pkg/admin/auth" + "github.com/streamnative/pulsar-admin-go/pkg/admin/config" ) func TestPulsarClientEndpointEscapes(t *testing.T) { - client := pulsarClient{Client: nil, APIVersion: V2} + client := pulsarClient{Client: nil, APIVersion: config.V2} actual := client.endpoint("/myendpoint", "abc%? /def", "ghi") expected := "/admin/v2/myendpoint/abc%25%3F%20%2Fdef/ghi" assert.Equal(t, expected, actual) } func TestNew(t *testing.T) { - config := &Config{} + config := &config.Config{} admin, err := New(config) require.NoError(t, err) require.NotNil(t, admin) } func TestNewWithAuthProvider(t *testing.T) { - config := &Config{} + config := &config.Config{} - tokenAuth, err := NewAuthenticationToken("eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9."+ + tokenAuth, err := auth.NewAuthenticationToken("eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9."+ "eyJzdWIiOiJhZG1pbiIsImlhdCI6MTUxNjIzOTAyMn0.sVt6cyu3HKd89LcQvZVMNbqT0DTl3FvG9oYbj8hBDqU", nil) require.NoError(t, err) require.NotNil(t, tokenAuth) @@ -56,7 +59,7 @@ type customAuthProvider struct { transport http.RoundTripper } -var _ AuthProvider = &customAuthProvider{} +var _ auth.Provider = &customAuthProvider{} func (c *customAuthProvider) RoundTrip(req *http.Request) (*http.Response, error) { panic("implement me") @@ -71,8 +74,8 @@ func (c *customAuthProvider) WithTransport(transport http.RoundTripper) { } func TestNewWithCustomAuthProviderWithTransport(t *testing.T) { - config := &Config{} - defaultTransport, err := NewDefaultTransport(config) + config := &config.Config{} + defaultTransport, err := auth.NewDefaultTransport(config) require.NoError(t, err) customAuthProvider := &customAuthProvider{ @@ -88,7 +91,7 @@ func TestNewWithCustomAuthProviderWithTransport(t *testing.T) { } func TestNewWithTlsAllowInsecure(t *testing.T) { - config := &Config{ + config := &config.Config{ TLSAllowInsecureConnection: true, } admin, err := New(config) diff --git a/pulsaradmin/pkg/admin/oauth2.go b/pulsaradmin/pkg/admin/auth/oauth2.go similarity index 99% rename from pulsaradmin/pkg/admin/oauth2.go rename to pulsaradmin/pkg/admin/auth/oauth2.go index c12d5d91f0..44677c384c 100644 --- a/pulsaradmin/pkg/admin/oauth2.go +++ b/pulsaradmin/pkg/admin/auth/oauth2.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package admin +package auth import ( "encoding/json" @@ -71,7 +71,7 @@ func NewAuthenticationOAuth2(issuer oauth2.Issuer, store store.Store) (*OAuth2Pr } // NewAuthenticationOAuth2WithDefaultFlow uses memory to save the grant -func NewAuthenticationOAuth2WithDefaultFlow(issuer oauth2.Issuer, keyFile string) (AuthProvider, error) { +func NewAuthenticationOAuth2WithDefaultFlow(issuer oauth2.Issuer, keyFile string) (Provider, error) { st := store.NewMemoryStore() flow, err := oauth2.NewDefaultClientCredentialsFlow(oauth2.ClientCredentialsFlowOptions{ KeyFile: keyFile, diff --git a/pulsaradmin/pkg/admin/oauth2_test.go b/pulsaradmin/pkg/admin/auth/oauth2_test.go similarity index 99% rename from pulsaradmin/pkg/admin/oauth2_test.go rename to pulsaradmin/pkg/admin/auth/oauth2_test.go index 590f609f56..f832af58cc 100644 --- a/pulsaradmin/pkg/admin/oauth2_test.go +++ b/pulsaradmin/pkg/admin/auth/oauth2_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package admin +package auth import ( "fmt" diff --git a/pulsaradmin/pkg/admin/auth_provider.go b/pulsaradmin/pkg/admin/auth/provider.go similarity index 67% rename from pulsaradmin/pkg/admin/auth_provider.go rename to pulsaradmin/pkg/admin/auth/provider.go index e04e7ea747..3ab20384ca 100644 --- a/pulsaradmin/pkg/admin/auth_provider.go +++ b/pulsaradmin/pkg/admin/auth/provider.go @@ -15,28 +15,23 @@ // specific language governing permissions and limitations // under the License. -package admin +package auth import ( - "crypto/tls" - "crypto/x509" - "io/ioutil" "net/http" + + "github.com/streamnative/pulsar-admin-go/pkg/admin/config" ) -// AuthProvider provide a general method to add auth message -type AuthProvider interface { +// Provider provide a general method to add auth message +type Provider interface { RoundTrip(req *http.Request) (*http.Response, error) Transport() http.RoundTripper WithTransport(tripper http.RoundTripper) } -type Transport struct { - T http.RoundTripper -} - -func GetAuthProvider(config *Config) (AuthProvider, error) { - var provider AuthProvider +func GetAuthProvider(config *config.Config) (Provider, error) { + var provider Provider defaultTransport, err := NewDefaultTransport(config) if err != nil { return nil, err @@ -69,32 +64,3 @@ func GetAuthProvider(config *Config) (AuthProvider, error) { } return provider, err } - -// GetDefaultTransport gets a default transport. -// Deprecated: Use NewDefaultTransport instead. -func GetDefaultTransport(config *Config) http.RoundTripper { - transport, err := NewDefaultTransport(config) - if err != nil { - panic(err) - } - - return transport -} - -func NewDefaultTransport(config *Config) (http.RoundTripper, error) { - transport := http.DefaultTransport.(*http.Transport).Clone() - tlsConfig := &tls.Config{ - InsecureSkipVerify: config.TLSAllowInsecureConnection, - } - if len(config.TLSTrustCertsFilePath) > 0 { - rootCA, err := ioutil.ReadFile(config.TLSTrustCertsFilePath) - if err != nil { - return nil, err - } - tlsConfig.RootCAs = x509.NewCertPool() - tlsConfig.RootCAs.AppendCertsFromPEM(rootCA) - } - transport.MaxIdleConnsPerHost = 10 - transport.TLSClientConfig = tlsConfig - return transport, nil -} diff --git a/pulsaradmin/pkg/admin/auth_tls.go b/pulsaradmin/pkg/admin/auth/tls.go similarity index 99% rename from pulsaradmin/pkg/admin/auth_tls.go rename to pulsaradmin/pkg/admin/auth/tls.go index d9473839f3..7c11fe27e7 100644 --- a/pulsaradmin/pkg/admin/auth_tls.go +++ b/pulsaradmin/pkg/admin/auth/tls.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package admin +package auth import ( "crypto/tls" diff --git a/pulsaradmin/pkg/admin/auth_token.go b/pulsaradmin/pkg/admin/auth/token.go similarity index 92% rename from pulsaradmin/pkg/admin/auth_token.go rename to pulsaradmin/pkg/admin/auth/token.go index 55fcd7170b..5b6eed6e8e 100644 --- a/pulsaradmin/pkg/admin/auth_token.go +++ b/pulsaradmin/pkg/admin/auth/token.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package admin +package auth import ( "encoding/json" @@ -34,7 +34,7 @@ const ( TokePluginShortName = "token" ) -type AuthToken struct { +type Token struct { Token string `json:"token"` } @@ -43,7 +43,7 @@ type TokenAuthProvider struct { token string } -// NewAuthenticationToken return a interface of AuthProvider with a string token. +// NewAuthenticationToken return a interface of Provider with a string token. func NewAuthenticationToken(token string, transport http.RoundTripper) (*TokenAuthProvider, error) { if len(token) == 0 { return nil, errors.New("No token provided") @@ -51,7 +51,7 @@ func NewAuthenticationToken(token string, transport http.RoundTripper) (*TokenAu return &TokenAuthProvider{token: token, T: transport}, nil } -// NewAuthenticationTokenFromFile return a interface of a AuthProvider with a string token file path. +// NewAuthenticationTokenFromFile return a interface of a Provider with a string token file path. func NewAuthenticationTokenFromFile(tokenFilePath string, transport http.RoundTripper) (*TokenAuthProvider, error) { data, err := ioutil.ReadFile(tokenFilePath) if err != nil { @@ -66,7 +66,7 @@ func NewAuthenticationTokenFromAuthParams(encodedAuthParam string, var tokenAuthProvider *TokenAuthProvider var err error - var tokenJSON AuthToken + var tokenJSON Token err = json.Unmarshal([]byte(encodedAuthParam), &tokenJSON) if err != nil { switch { diff --git a/pulsaradmin/pkg/admin/auth/transport.go b/pulsaradmin/pkg/admin/auth/transport.go new file mode 100644 index 0000000000..4527e2a5dc --- /dev/null +++ b/pulsaradmin/pkg/admin/auth/transport.go @@ -0,0 +1,60 @@ +// 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 auth + +import ( + "crypto/tls" + "crypto/x509" + "io/ioutil" + "net/http" + + "github.com/streamnative/pulsar-admin-go/pkg/admin/config" +) + +type Transport struct { + T http.RoundTripper +} + +// GetDefaultTransport gets a default transport. +// Deprecated: Use NewDefaultTransport instead. +func GetDefaultTransport(config *config.Config) http.RoundTripper { + transport, err := NewDefaultTransport(config) + if err != nil { + panic(err) + } + + return transport +} + +func NewDefaultTransport(config *config.Config) (http.RoundTripper, error) { + transport := http.DefaultTransport.(*http.Transport).Clone() + tlsConfig := &tls.Config{ + InsecureSkipVerify: config.TLSAllowInsecureConnection, + } + if len(config.TLSTrustCertsFilePath) > 0 { + rootCA, err := ioutil.ReadFile(config.TLSTrustCertsFilePath) + if err != nil { + return nil, err + } + tlsConfig.RootCAs = x509.NewCertPool() + tlsConfig.RootCAs.AppendCertsFromPEM(rootCA) + } + transport.MaxIdleConnsPerHost = 10 + transport.TLSClientConfig = tlsConfig + return transport, nil +} diff --git a/pulsaradmin/pkg/admin/api_version.go b/pulsaradmin/pkg/admin/config/api_version.go similarity index 98% rename from pulsaradmin/pkg/admin/api_version.go rename to pulsaradmin/pkg/admin/config/api_version.go index 2347e864bf..95f6704580 100644 --- a/pulsaradmin/pkg/admin/api_version.go +++ b/pulsaradmin/pkg/admin/config/api_version.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package admin +package config type APIVersion int diff --git a/pulsaradmin/pkg/admin/api_version_test.go b/pulsaradmin/pkg/admin/config/api_version_test.go similarity index 98% rename from pulsaradmin/pkg/admin/api_version_test.go rename to pulsaradmin/pkg/admin/config/api_version_test.go index 094fbd768b..e1dc7bdb69 100644 --- a/pulsaradmin/pkg/admin/api_version_test.go +++ b/pulsaradmin/pkg/admin/config/api_version_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package admin +package config import ( "testing" diff --git a/pulsaradmin/pkg/admin/config.go b/pulsaradmin/pkg/admin/config/config.go similarity index 99% rename from pulsaradmin/pkg/admin/config.go rename to pulsaradmin/pkg/admin/config/config.go index 2639fbc139..9428f3a909 100644 --- a/pulsaradmin/pkg/admin/config.go +++ b/pulsaradmin/pkg/admin/config/config.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package admin +package config type Config struct { // the web service url that pulsarctl connects to. Default is http://localhost:8080 From eca3965b5a97b976e4a3f598724061f9cba3a280 Mon Sep 17 00:00:00 2001 From: Max Xu Date: Thu, 16 Mar 2023 11:25:05 +0800 Subject: [PATCH 242/348] Add alias package (streamnative/pulsar-admin-go#5) feat: add alias package Signed-off-by: Max Xu --- pulsaradmin/alias.go | 34 ++++++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 pulsaradmin/alias.go diff --git a/pulsaradmin/alias.go b/pulsaradmin/alias.go new file mode 100644 index 0000000000..a114608803 --- /dev/null +++ b/pulsaradmin/alias.go @@ -0,0 +1,34 @@ +// 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 pulsaradmin + +import ( + "github.com/streamnative/pulsar-admin-go/pkg/admin" + "github.com/streamnative/pulsar-admin-go/pkg/admin/config" +) + +// Client contains all admin interfaces for operating pulsar resources +type Client = admin.Client + +// Config are the arguments for creating a new admin Client +type Config = config.Config + +var ( + // NewClient returns a new admin Client for operating pulsar resources + NewClient = admin.New +) From 8c65193e9be91e053cf630fea9374d97ca99b445 Mon Sep 17 00:00:00 2001 From: Max Xu Date: Thu, 16 Mar 2023 15:10:28 +0800 Subject: [PATCH 243/348] fix: golangci-lint errors (streamnative/pulsar-admin-go#7) fix: resolve lint errors Signed-off-by: Max Xu --- pulsaradmin/pkg/admin/functions.go | 4 +++- pulsaradmin/pkg/admin/subscription.go | 2 +- pulsaradmin/pkg/admin/token.go | 2 +- pulsaradmin/pkg/algorithm/hmac/hs256.go | 4 +++- pulsaradmin/pkg/algorithm/hmac/hs384.go | 4 +++- pulsaradmin/pkg/algorithm/hmac/hs512.go | 4 +++- pulsaradmin/pkg/rest/client.go | 12 +++++++----- pulsaradmin/pkg/utils/message.go | 2 +- pulsaradmin/pkg/utils/namespace_name.go | 2 +- 9 files changed, 23 insertions(+), 13 deletions(-) diff --git a/pulsaradmin/pkg/admin/functions.go b/pulsaradmin/pkg/admin/functions.go index dd31d085bf..aae373d609 100644 --- a/pulsaradmin/pkg/admin/functions.go +++ b/pulsaradmin/pkg/admin/functions.go @@ -675,7 +675,9 @@ func (f *functions) Upload(sourceFile, path string) error { if err != nil { return err } - w.WriteField("path", path) + if err := w.WriteField("path", path); err != nil { + return err + } err = w.Close() if err != nil { return err diff --git a/pulsaradmin/pkg/admin/subscription.go b/pulsaradmin/pkg/admin/subscription.go index a3c240f3c3..3f7c30b3c6 100644 --- a/pulsaradmin/pkg/admin/subscription.go +++ b/pulsaradmin/pkg/admin/subscription.go @@ -27,7 +27,7 @@ import ( "strconv" "strings" - "github.com/golang/protobuf/proto" + "github.com/golang/protobuf/proto" //nolint:staticcheck "github.com/streamnative/pulsar-admin-go/pkg/utils" ) diff --git a/pulsaradmin/pkg/admin/token.go b/pulsaradmin/pkg/admin/token.go index 7d2c20428b..371711a571 100644 --- a/pulsaradmin/pkg/admin/token.go +++ b/pulsaradmin/pkg/admin/token.go @@ -106,7 +106,7 @@ func (t *token) CreateToken( headers map[string]interface{}) (string, error) { signMethod := parseAlgorithmToJwtSignMethod(algorithm) tokenString := jwt.NewWithClaims(signMethod, mapClaims) - if headers != nil && len(headers) > 0 { + if len(headers) > 0 { for s, i := range headers { tokenString.Header[s] = i } diff --git a/pulsaradmin/pkg/algorithm/hmac/hs256.go b/pulsaradmin/pkg/algorithm/hmac/hs256.go index 341df6b00f..0e2644e6e4 100644 --- a/pulsaradmin/pkg/algorithm/hmac/hs256.go +++ b/pulsaradmin/pkg/algorithm/hmac/hs256.go @@ -31,7 +31,9 @@ type HS256 struct{} func (h *HS256) GenerateSecret() ([]byte, error) { bytes := make([]byte, 32) - rand.Read(bytes) + if _, err := rand.Read(bytes); err != nil { + return nil, err + } s := hmac.New(sha256.New, bytes) return s.Sum(nil), nil } diff --git a/pulsaradmin/pkg/algorithm/hmac/hs384.go b/pulsaradmin/pkg/algorithm/hmac/hs384.go index 4ff2ad2e75..a3127985a4 100644 --- a/pulsaradmin/pkg/algorithm/hmac/hs384.go +++ b/pulsaradmin/pkg/algorithm/hmac/hs384.go @@ -31,7 +31,9 @@ type HS384 struct{} func (h *HS384) GenerateSecret() ([]byte, error) { bytes := make([]byte, 48) - rand.Read(bytes) + if _, err := rand.Read(bytes); err != nil { + return nil, err + } s := hmac.New(sha512.New384, bytes) return s.Sum(nil), nil } diff --git a/pulsaradmin/pkg/algorithm/hmac/hs512.go b/pulsaradmin/pkg/algorithm/hmac/hs512.go index 761d43e60c..21ff5fc903 100644 --- a/pulsaradmin/pkg/algorithm/hmac/hs512.go +++ b/pulsaradmin/pkg/algorithm/hmac/hs512.go @@ -31,7 +31,9 @@ type HS512 struct{} func (h *HS512) GenerateSecret() ([]byte, error) { bytes := make([]byte, 64) - rand.Read(bytes) + if _, err := rand.Read(bytes); err != nil { + return nil, err + } s := hmac.New(sha512.New, bytes) return s.Sum(nil), nil } diff --git a/pulsaradmin/pkg/rest/client.go b/pulsaradmin/pkg/rest/client.go index 10ac5473bc..64af1a0e78 100644 --- a/pulsaradmin/pkg/rest/client.go +++ b/pulsaradmin/pkg/rest/client.go @@ -120,6 +120,7 @@ func (c *Client) GetWithOptions(endpoint string, obj interface{}, params map[str req.params = query } + //nolint:bodyclose resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return nil, err @@ -174,6 +175,7 @@ func (c *Client) PutWithQueryParams(endpoint string, in, obj interface{}, params req.params = query } + //nolint:bodyclose resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return err @@ -197,7 +199,7 @@ func (c *Client) PutWithMultiPart(endpoint string, body io.Reader, contentType s req.body = body req.contentType = contentType - // nolint + //nolint resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return err @@ -225,7 +227,7 @@ func (c *Client) DeleteWithQueryParams(endpoint string, params map[string]string req.params = query } - // nolint + //nolint resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return err @@ -246,7 +248,7 @@ func (c *Client) PostWithObj(endpoint string, in, obj interface{}) error { } req.obj = in - // nolint + //nolint resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return err @@ -270,7 +272,7 @@ func (c *Client) PostWithMultiPart(endpoint string, in interface{}, body io.Read req.body = body req.contentType = contentType - // nolint + //nolint resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return err @@ -295,7 +297,7 @@ func (c *Client) PostWithQueryParams(endpoint string, in interface{}, params map } req.params = query } - // nolint + //nolint resp, err := checkSuccessful(c.doRequest(req)) if err != nil { return err diff --git a/pulsaradmin/pkg/utils/message.go b/pulsaradmin/pkg/utils/message.go index 6ab1a5cfa4..2f0e4befe8 100644 --- a/pulsaradmin/pkg/utils/message.go +++ b/pulsaradmin/pkg/utils/message.go @@ -17,7 +17,7 @@ package utils -// nolint +//nolint import ( "github.com/golang/protobuf/proto" ) diff --git a/pulsaradmin/pkg/utils/namespace_name.go b/pulsaradmin/pkg/utils/namespace_name.go index 59fb8d4796..11b7435545 100644 --- a/pulsaradmin/pkg/utils/namespace_name.go +++ b/pulsaradmin/pkg/utils/namespace_name.go @@ -81,7 +81,7 @@ func validateNamespaceName(tenant, namespace string) error { // allowed characters for property, namespace, cluster and topic // names are alphanumeric (a-zA-Z0-9) and these special chars -=:. // and % is allowed as part of valid URL encoding -var patten = regexp.MustCompile("^[-=:.\\w]*$") +var patten = regexp.MustCompile(`^[-=:.\w]*$`) func CheckName(name string) bool { return patten.MatchString(name) From 735e3c3cedd84ef1b32134222d1875c1e0e7e5f2 Mon Sep 17 00:00:00 2001 From: Max Xu Date: Thu, 16 Mar 2023 15:13:15 +0800 Subject: [PATCH 244/348] ci: add lint and test (streamnative/pulsar-admin-go#6) ci: add lint and test CI Signed-off-by: Max Xu --- pulsaradmin/.github/workflows/ci.yaml | 58 +++++++++++++++++++++++++++ pulsaradmin/.golangci.yaml | 42 +++++++++++++++++++ 2 files changed, 100 insertions(+) create mode 100644 pulsaradmin/.github/workflows/ci.yaml create mode 100644 pulsaradmin/.golangci.yaml diff --git a/pulsaradmin/.github/workflows/ci.yaml b/pulsaradmin/.github/workflows/ci.yaml new file mode 100644 index 0000000000..0fc45a3ba6 --- /dev/null +++ b/pulsaradmin/.github/workflows/ci.yaml @@ -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. +# + +name: CI + +on: + pull_request: + branches: + - '*' + +jobs: + lint: + runs-on: ubuntu-latest + steps: + - name: Checkout + uses: actions/checkout@v3 + + - name: Setup Go + uses: actions/setup-go@v3 + with: + go-version: 1.18 + + - name: Run golangci-lint + uses: golangci/golangci-lint-action@v3 + with: + version: v1.51.2 + + test: + runs-on: ubuntu-latest + steps: + - name: Checkout + uses: actions/checkout@v3 + + - name: Setup Go + uses: actions/setup-go@v3 + with: + go-version: 1.18 + + - name: Run build and test + run: | + go build ./... + go test -race ./... + diff --git a/pulsaradmin/.golangci.yaml b/pulsaradmin/.golangci.yaml new file mode 100644 index 0000000000..292947d170 --- /dev/null +++ b/pulsaradmin/.golangci.yaml @@ -0,0 +1,42 @@ +# +# 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. +# + +run: + deadline: 6m + +linters: + # disable all for explicit enable + disable-all: true + enable: + - errcheck + - gosimple + - govet + - ineffassign + - staticcheck + - typecheck + - unused + - lll + - goimports + - bodyclose + - misspell + - prealloc + - revive + - stylecheck + - unconvert + - unparam From 49c38ec4f46582a9491e28e0ff6ced0ea467ad72 Mon Sep 17 00:00:00 2001 From: Max Xu Date: Thu, 16 Mar 2023 22:37:29 +0800 Subject: [PATCH 245/348] chore: clean non-admin apis (streamnative/pulsar-admin-go#9) Signed-off-by: Max Xu --- pulsaradmin/go.mod | 1 - pulsaradmin/go.sum | 2 - pulsaradmin/pkg/admin/admin.go | 1 - pulsaradmin/pkg/admin/token.go | 190 ------------------- pulsaradmin/pkg/admin/token_test.go | 62 ------ pulsaradmin/pkg/algorithm/algorithm.go | 75 -------- pulsaradmin/pkg/algorithm/algorithm_test.go | 141 -------------- pulsaradmin/pkg/algorithm/ecdsa/es256.go | 42 ---- pulsaradmin/pkg/algorithm/ecdsa/es384.go | 42 ---- pulsaradmin/pkg/algorithm/ecdsa/es512.go | 42 ---- pulsaradmin/pkg/algorithm/hmac/hs256.go | 43 ----- pulsaradmin/pkg/algorithm/hmac/hs384.go | 43 ----- pulsaradmin/pkg/algorithm/hmac/hs512.go | 43 ----- pulsaradmin/pkg/algorithm/keypair/keypair.go | 138 -------------- pulsaradmin/pkg/algorithm/rsa/rs256.go | 41 ---- pulsaradmin/pkg/algorithm/rsa/rs384.go | 41 ---- pulsaradmin/pkg/algorithm/rsa/rs512.go | 41 ---- 17 files changed, 988 deletions(-) delete mode 100644 pulsaradmin/pkg/admin/token.go delete mode 100644 pulsaradmin/pkg/admin/token_test.go delete mode 100644 pulsaradmin/pkg/algorithm/algorithm.go delete mode 100644 pulsaradmin/pkg/algorithm/algorithm_test.go delete mode 100644 pulsaradmin/pkg/algorithm/ecdsa/es256.go delete mode 100644 pulsaradmin/pkg/algorithm/ecdsa/es384.go delete mode 100644 pulsaradmin/pkg/algorithm/ecdsa/es512.go delete mode 100644 pulsaradmin/pkg/algorithm/hmac/hs256.go delete mode 100644 pulsaradmin/pkg/algorithm/hmac/hs384.go delete mode 100644 pulsaradmin/pkg/algorithm/hmac/hs512.go delete mode 100644 pulsaradmin/pkg/algorithm/keypair/keypair.go delete mode 100644 pulsaradmin/pkg/algorithm/rsa/rs256.go delete mode 100644 pulsaradmin/pkg/algorithm/rsa/rs384.go delete mode 100644 pulsaradmin/pkg/algorithm/rsa/rs512.go diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod index b7b44c1a75..a8ec585fc7 100644 --- a/pulsaradmin/go.mod +++ b/pulsaradmin/go.mod @@ -5,7 +5,6 @@ go 1.18 require ( github.com/99designs/keyring v1.2.1 github.com/apache/pulsar-client-go v0.9.0 - github.com/golang-jwt/jwt/v4 v4.4.3 github.com/golang/protobuf v1.5.2 github.com/pkg/errors v0.9.1 github.com/stretchr/testify v1.8.0 diff --git a/pulsaradmin/go.sum b/pulsaradmin/go.sum index 75390df360..124611f102 100644 --- a/pulsaradmin/go.sum +++ b/pulsaradmin/go.sum @@ -116,8 +116,6 @@ github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7a github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang-jwt/jwt v3.2.1+incompatible h1:73Z+4BJcrTC+KczS6WvTPvRGOp1WmfEP4Q1lOd9Z/+c= github.com/golang-jwt/jwt v3.2.1+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= -github.com/golang-jwt/jwt/v4 v4.4.3 h1:Hxl6lhQFj4AnOX6MLrsCb/+7tCj7DxP7VA+2rDIq5AU= -github.com/golang-jwt/jwt/v4 v4.4.3/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= diff --git a/pulsaradmin/pkg/admin/admin.go b/pulsaradmin/pkg/admin/admin.go index 8931d7dd81..84b5760bf5 100644 --- a/pulsaradmin/pkg/admin/admin.go +++ b/pulsaradmin/pkg/admin/admin.go @@ -56,7 +56,6 @@ type Client interface { BrokerStats() BrokerStats ResourceQuotas() ResourceQuotas FunctionsWorker() FunctionsWorker - Token() Token Packages() Packages } diff --git a/pulsaradmin/pkg/admin/token.go b/pulsaradmin/pkg/admin/token.go deleted file mode 100644 index 371711a571..0000000000 --- a/pulsaradmin/pkg/admin/token.go +++ /dev/null @@ -1,190 +0,0 @@ -// 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 admin - -import ( - "encoding/base64" - "strings" - "time" - - "github.com/golang-jwt/jwt/v4" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm" - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" - - "github.com/pkg/errors" -) - -type Token interface { - // CreateKeyPair is used to create public and private key pair using the given signature algorithm - CreateKeyPair(algorithm.Algorithm) (*keypair.KeyPair, error) - - // CreateSecretKey is used for creating a secret key - CreateSecretKey(algorithm.Algorithm) ([]byte, error) - - // Create creates a token object using the specified signature algorithm, private key, - // object and the expire time - Create(algorithm.Algorithm, interface{}, string, int64) (string, error) - - // CreateToken creates a token object using the specified signature algorithm, private key - // custom claim and header - CreateToken(algorithm.Algorithm, interface{}, *jwt.MapClaims, map[string]interface{}) (string, error) - - // Validate a token is valid or not - Validate(algorithm.Algorithm, string, interface{}) (string, int64, error) - - // GetAlgorithm gets which algorithm the token used - GetAlgorithm(string) (string, error) - - // GetSubject gets the subject of a token - GetSubject(string) (string, error) -} - -type token struct { - pulsar *pulsarClient -} - -func (c *pulsarClient) Token() Token { - return &token{ - pulsar: c, - } -} - -func (t *token) CreateKeyPair(signatureAlgorithm algorithm.Algorithm) (*keypair.KeyPair, error) { - sa, err := algorithm.GetSignatureAlgorithm(signatureAlgorithm) - if err != nil { - return nil, err - } - return sa.GenerateKeyPair() -} - -func (t *token) CreateSecretKey(signatureAlgorithm algorithm.Algorithm) ([]byte, error) { - sa, err := algorithm.GetSignatureAlgorithm(signatureAlgorithm) - if err != nil { - return nil, err - } - return sa.GenerateSecret() -} - -func (t *token) Create(algorithm algorithm.Algorithm, signKey interface{}, subject string, - expireTime int64) (string, error) { - - var claims *jwt.MapClaims - if expireTime <= 0 { - claims = &jwt.MapClaims{ - "sub": subject, - } - } else { - claims = &jwt.MapClaims{ - "sub": subject, - "exp": jwt.NewNumericDate(time.Unix(expireTime, 0)), - } - } - - return t.CreateToken(algorithm, signKey, claims, nil) -} - -func (t *token) CreateToken( - algorithm algorithm.Algorithm, - signKey interface{}, - mapClaims *jwt.MapClaims, - headers map[string]interface{}) (string, error) { - signMethod := parseAlgorithmToJwtSignMethod(algorithm) - tokenString := jwt.NewWithClaims(signMethod, mapClaims) - if len(headers) > 0 { - for s, i := range headers { - tokenString.Header[s] = i - } - } - return tokenString.SignedString(signKey) -} - -func (t *token) Validate(algorithm algorithm.Algorithm, tokenString string, - signKey interface{}) (string, int64, error) { - - // verify the signature algorithm - parsedToken, err := jwt.ParseWithClaims(tokenString, &jwt.RegisteredClaims{}, - func(jt *jwt.Token) (i interface{}, e error) { - signMethod := parseAlgorithmToJwtSignMethod(algorithm) - if jt.Method != signMethod { - return nil, errors.Errorf("unexpected signing method: %s", algorithm) - } - return signKey, nil - }) - - // get the subject and the expire time - if claim, ok := parsedToken.Claims.(*jwt.RegisteredClaims); parsedToken.Valid && ok { - expiresAt := claim.ExpiresAt - exp := int64(0) - if expiresAt != nil { - exp = expiresAt.Unix() - } - return claim.Subject, exp, nil - } - - return "", 0, err -} - -func (t *token) GetAlgorithm(tokenString string) (string, error) { - parts := strings.Split(tokenString, ".") - alg, err := base64.RawURLEncoding.DecodeString(parts[0]) - if err != nil { - return "", err - } - return string(alg), nil -} - -func (t *token) GetSubject(tokenString string) (string, error) { - parts := strings.Split(tokenString, ".") - alg, err := base64.RawURLEncoding.DecodeString(parts[1]) - if err != nil { - return "", err - } - return string(alg), nil -} - -func parseAlgorithmToJwtSignMethod(a algorithm.Algorithm) jwt.SigningMethod { - switch a { - case algorithm.HS256: - return jwt.SigningMethodHS256 - case algorithm.HS384: - return jwt.SigningMethodHS384 - case algorithm.HS512: - return jwt.SigningMethodHS512 - case algorithm.RS256: - return jwt.SigningMethodRS256 - case algorithm.RS384: - return jwt.SigningMethodRS384 - case algorithm.RS512: - return jwt.SigningMethodRS512 - case algorithm.ES256: - return jwt.SigningMethodES256 - case algorithm.ES384: - return jwt.SigningMethodES384 - case algorithm.ES512: - return jwt.SigningMethodES512 - case algorithm.PS256: - return jwt.SigningMethodPS256 - case algorithm.PS384: - return jwt.SigningMethodPS384 - case algorithm.PS512: - return jwt.SigningMethodPS512 - default: - return jwt.SigningMethodRS256 - } -} diff --git a/pulsaradmin/pkg/admin/token_test.go b/pulsaradmin/pkg/admin/token_test.go deleted file mode 100644 index d8fe91c933..0000000000 --- a/pulsaradmin/pkg/admin/token_test.go +++ /dev/null @@ -1,62 +0,0 @@ -// 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 admin - -import ( - "testing" - "time" - - "github.com/stretchr/testify/require" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm" -) - -func TestCreateToken(t *testing.T) { - tokenProvider := token{} - - alg := algorithm.HS256 - key, err := tokenProvider.CreateSecretKey(alg) - require.NoError(t, err) - - subject := "test-role" - myToken, err := tokenProvider.Create(alg, key, subject, 0) - require.NoError(t, err) - - parsedSubject, exp, err := tokenProvider.Validate(alg, myToken, key) - require.NoError(t, err) - require.Equal(t, subject, parsedSubject) - require.Equal(t, exp, int64(0)) -} - -func TestCreateTokenWithExp(t *testing.T) { - tokenProvider := token{} - - alg := algorithm.HS256 - key, err := tokenProvider.CreateSecretKey(alg) - require.NoError(t, err) - - subject := "test-role" - exp := time.Now().Add(time.Hour).Unix() - myToken, err := tokenProvider.Create(alg, key, subject, exp) - require.NoError(t, err) - - parsedSubject, exp, err := tokenProvider.Validate(alg, myToken, key) - require.NoError(t, err) - require.Equal(t, subject, parsedSubject) - require.Equal(t, exp, exp) -} diff --git a/pulsaradmin/pkg/algorithm/algorithm.go b/pulsaradmin/pkg/algorithm/algorithm.go deleted file mode 100644 index bd66da531e..0000000000 --- a/pulsaradmin/pkg/algorithm/algorithm.go +++ /dev/null @@ -1,75 +0,0 @@ -// 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 algorithm - -import ( - "github.com/pkg/errors" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/ecdsa" - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/hmac" - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/rsa" -) - -type Algorithm string - -const ( - HS256 Algorithm = "HS256" - HS384 Algorithm = "HS384" - HS512 Algorithm = "HS512" - RS256 Algorithm = "RS256" - RS384 Algorithm = "RS384" - RS512 Algorithm = "RS512" - ES256 Algorithm = "ES256" - ES384 Algorithm = "ES384" - ES512 Algorithm = "ES512" - PS256 Algorithm = "PS256" - PS384 Algorithm = "PS384" - PS512 Algorithm = "PS512" -) - -var algorithmMap = map[Algorithm]SignatureAlgorithm{ - HS256: new(hmac.HS256), - HS384: new(hmac.HS384), - HS512: new(hmac.HS512), - RS256: new(rsa.RS256), - RS384: new(rsa.RS384), - RS512: new(rsa.RS512), - ES256: new(ecdsa.ES256), - ES384: new(ecdsa.ES384), - ES512: new(ecdsa.ES512), -} - -// SignatureAlgorithm is a collection of all signature algorithm and it provides -// some basic method to use -type SignatureAlgorithm interface { - // GenerateKeyPair generates public and private key - GenerateKeyPair() (*keypair.KeyPair, error) - - // GenerateSecret is used to generating a secret - GenerateSecret() ([]byte, error) -} - -func GetSignatureAlgorithm(algorithm Algorithm) (SignatureAlgorithm, error) { - sa := algorithmMap[algorithm] - if sa == nil { - return nil, errors.Errorf("the signature algorithm '%s' is invalid. Valid options are: 'HS256', "+ - "'HS384', 'HS512', 'RS256', 'RS384', 'RS512', 'ES256', 'ES384', 'ES512'\n", algorithm) - } - return sa, nil -} diff --git a/pulsaradmin/pkg/algorithm/algorithm_test.go b/pulsaradmin/pkg/algorithm/algorithm_test.go deleted file mode 100644 index 252b80d595..0000000000 --- a/pulsaradmin/pkg/algorithm/algorithm_test.go +++ /dev/null @@ -1,141 +0,0 @@ -// 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 algorithm - -import ( - "fmt" - "testing" - - "github.com/stretchr/testify/assert" -) - -var testData = []struct { - name string - algorithm Algorithm -}{ - {"HMAC", HS256}, - {"HMAC", HS384}, - {"HMAC", HS512}, - {"RSA", RS256}, - {"RSA", RS384}, - {"RSA", RS512}, - {"ECDSA", ES256}, - {"ECDSA", ES384}, - {"ECDSA", ES512}, - {"INVALID", Algorithm("INVALID")}, -} - -func TestGetSignatureAlgorithm(t *testing.T) { - // test ci status - for _, data := range testData { - switch data.name { - case "HMAC": - testHMAC(t, data.algorithm) - case "RSA": - testRSA(t, data.algorithm) - case "ECDSA": - testECDSA(t, data.algorithm) - default: - sa, err := GetSignatureAlgorithm(data.algorithm) - assert.Nil(t, sa) - assert.NotNil(t, err) - assert.Equal(t, - fmt.Sprintf("the signature algorithm '%s' is invalid. Valid options are: 'HS256', "+ - "'HS384', 'HS512', 'RS256', 'RS384', 'RS512', 'ES256', 'ES384', 'ES512'\n", data.algorithm), - err.Error()) - } - } -} - -func testHMAC(t *testing.T, algorithm Algorithm) { - sa, err := GetSignatureAlgorithm(algorithm) - assert.Nil(t, err) - secrets, err := sa.GenerateSecret() - assert.Nil(t, err) - switch algorithm { - case HS256: - assert.Equal(t, 32, len(secrets)) - case HS384: - assert.Equal(t, 48, len(secrets)) - case HS512: - assert.Equal(t, 64, len(secrets)) - } - - kp, err := sa.GenerateKeyPair() - assert.Nil(t, kp) - assert.NotNil(t, err) - assert.Equal(t, "unsupported operation", err.Error()) -} - -func testRSA(t *testing.T, algorithm Algorithm) { - sa, err := GetSignatureAlgorithm(algorithm) - assert.Nil(t, err) - - kp, err := sa.GenerateKeyPair() - assert.Nil(t, err) - assert.NotNil(t, kp) - _, err = kp.EncodedPrivateKey() - assert.Nil(t, err) - _, err = kp.EncodedPublicKey() - assert.Nil(t, err) - - rsaPrivateKey, err := kp.GetRsaPrivateKey() - assert.Nil(t, err) - assert.NotNil(t, rsaPrivateKey) - - ecdsaPrivateKey, err := kp.GetEcdsaPrivateKey() - assert.Nil(t, ecdsaPrivateKey) - assert.NotNil(t, err) - assert.Equal(t, - "the private key is not generated using ECDSA signature algorithm", - err.Error()) - - secrets, err := sa.GenerateSecret() - assert.Nil(t, secrets) - assert.NotNil(t, err) - assert.Equal(t, "unsupported operation", err.Error()) -} - -func testECDSA(t *testing.T, algorithm Algorithm) { - sa, err := GetSignatureAlgorithm(algorithm) - assert.Nil(t, err) - - kp, err := sa.GenerateKeyPair() - assert.Nil(t, err) - assert.NotNil(t, kp) - _, err = kp.EncodedPrivateKey() - assert.Nil(t, err) - _, err = kp.EncodedPublicKey() - assert.Nil(t, err) - - ecdsaPrivateKey, err := kp.GetEcdsaPrivateKey() - assert.Nil(t, err) - assert.NotNil(t, ecdsaPrivateKey) - - rsaPrivateKey, err := kp.GetRsaPrivateKey() - assert.Nil(t, rsaPrivateKey) - assert.NotNil(t, err) - assert.Equal(t, - "the private key is not generated using RSA signature algorithm", - err.Error()) - - secrets, err := sa.GenerateSecret() - assert.Nil(t, secrets) - assert.NotNil(t, err) - assert.Equal(t, "unsupported operation", err.Error()) -} diff --git a/pulsaradmin/pkg/algorithm/ecdsa/es256.go b/pulsaradmin/pkg/algorithm/ecdsa/es256.go deleted file mode 100644 index f6334554bc..0000000000 --- a/pulsaradmin/pkg/algorithm/ecdsa/es256.go +++ /dev/null @@ -1,42 +0,0 @@ -// 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 ecdsa - -import ( - "crypto/ecdsa" - "crypto/elliptic" - "crypto/rand" - - "github.com/pkg/errors" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" -) - -type ES256 struct{} - -func (h *ES256) GenerateSecret() ([]byte, error) { - return nil, errors.New("unsupported operation") -} - -func (h *ES256) GenerateKeyPair() (*keypair.KeyPair, error) { - pri, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader) - if err != nil { - return nil, err - } - return keypair.New(keypair.ECDSA, pri), nil -} diff --git a/pulsaradmin/pkg/algorithm/ecdsa/es384.go b/pulsaradmin/pkg/algorithm/ecdsa/es384.go deleted file mode 100644 index fe6d3c48eb..0000000000 --- a/pulsaradmin/pkg/algorithm/ecdsa/es384.go +++ /dev/null @@ -1,42 +0,0 @@ -// 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 ecdsa - -import ( - "crypto/ecdsa" - "crypto/elliptic" - "crypto/rand" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" - - "github.com/pkg/errors" -) - -type ES384 struct{} - -func (h *ES384) GenerateSecret() ([]byte, error) { - return nil, errors.New("unsupported operation") -} - -func (h *ES384) GenerateKeyPair() (*keypair.KeyPair, error) { - pri, err := ecdsa.GenerateKey(elliptic.P384(), rand.Reader) - if err != nil { - return nil, err - } - return keypair.New(keypair.ECDSA, pri), nil -} diff --git a/pulsaradmin/pkg/algorithm/ecdsa/es512.go b/pulsaradmin/pkg/algorithm/ecdsa/es512.go deleted file mode 100644 index 64d7498d0e..0000000000 --- a/pulsaradmin/pkg/algorithm/ecdsa/es512.go +++ /dev/null @@ -1,42 +0,0 @@ -// 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 ecdsa - -import ( - "crypto/ecdsa" - "crypto/elliptic" - "crypto/rand" - - "github.com/pkg/errors" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" -) - -type ES512 struct{} - -func (h *ES512) GenerateSecret() ([]byte, error) { - return nil, errors.New("unsupported operation") -} - -func (h *ES512) GenerateKeyPair() (*keypair.KeyPair, error) { - pri, err := ecdsa.GenerateKey(elliptic.P521(), rand.Reader) - if err != nil { - return nil, err - } - return keypair.New(keypair.ECDSA, pri), nil -} diff --git a/pulsaradmin/pkg/algorithm/hmac/hs256.go b/pulsaradmin/pkg/algorithm/hmac/hs256.go deleted file mode 100644 index 0e2644e6e4..0000000000 --- a/pulsaradmin/pkg/algorithm/hmac/hs256.go +++ /dev/null @@ -1,43 +0,0 @@ -// 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 hmac - -import ( - "crypto/hmac" - "crypto/rand" - "crypto/sha256" - - "github.com/pkg/errors" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" -) - -type HS256 struct{} - -func (h *HS256) GenerateSecret() ([]byte, error) { - bytes := make([]byte, 32) - if _, err := rand.Read(bytes); err != nil { - return nil, err - } - s := hmac.New(sha256.New, bytes) - return s.Sum(nil), nil -} - -func (h *HS256) GenerateKeyPair() (*keypair.KeyPair, error) { - return nil, errors.New("unsupported operation") -} diff --git a/pulsaradmin/pkg/algorithm/hmac/hs384.go b/pulsaradmin/pkg/algorithm/hmac/hs384.go deleted file mode 100644 index a3127985a4..0000000000 --- a/pulsaradmin/pkg/algorithm/hmac/hs384.go +++ /dev/null @@ -1,43 +0,0 @@ -// 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 hmac - -import ( - "crypto/hmac" - "crypto/rand" - "crypto/sha512" - - "github.com/pkg/errors" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" -) - -type HS384 struct{} - -func (h *HS384) GenerateSecret() ([]byte, error) { - bytes := make([]byte, 48) - if _, err := rand.Read(bytes); err != nil { - return nil, err - } - s := hmac.New(sha512.New384, bytes) - return s.Sum(nil), nil -} - -func (h *HS384) GenerateKeyPair() (*keypair.KeyPair, error) { - return nil, errors.New("unsupported operation") -} diff --git a/pulsaradmin/pkg/algorithm/hmac/hs512.go b/pulsaradmin/pkg/algorithm/hmac/hs512.go deleted file mode 100644 index 21ff5fc903..0000000000 --- a/pulsaradmin/pkg/algorithm/hmac/hs512.go +++ /dev/null @@ -1,43 +0,0 @@ -// 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 hmac - -import ( - "crypto/hmac" - "crypto/rand" - "crypto/sha512" - - "github.com/pkg/errors" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" -) - -type HS512 struct{} - -func (h *HS512) GenerateSecret() ([]byte, error) { - bytes := make([]byte, 64) - if _, err := rand.Read(bytes); err != nil { - return nil, err - } - s := hmac.New(sha512.New, bytes) - return s.Sum(nil), nil -} - -func (h *HS512) GenerateKeyPair() (*keypair.KeyPair, error) { - return nil, errors.New("unsupported operation") -} diff --git a/pulsaradmin/pkg/algorithm/keypair/keypair.go b/pulsaradmin/pkg/algorithm/keypair/keypair.go deleted file mode 100644 index 2f6efb34de..0000000000 --- a/pulsaradmin/pkg/algorithm/keypair/keypair.go +++ /dev/null @@ -1,138 +0,0 @@ -// 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 keypair - -import ( - "crypto/ecdsa" - "crypto/rsa" - "crypto/x509" - "fmt" - - "github.com/pkg/errors" -) - -type KeyType string - -const ( - RSA KeyType = "RSA" - ECDSA KeyType = "ECDSA" -) - -// KeyPair saves the ecdsa private key or the rsa private key and provides -// a get public/private encoded bytes array method -type KeyPair struct { - keyType KeyType - privateKey interface{} -} - -func New(keyType KeyType, privateKey interface{}) *KeyPair { - return &KeyPair{ - keyType: keyType, - privateKey: privateKey, - } -} - -// EncodedPrivateKey gets the encoded private key -func (k *KeyPair) EncodedPrivateKey() ([]byte, error) { - return x509.MarshalPKCS8PrivateKey(k.privateKey) -} - -// DecodePrivateKey parses the private key to a KeyPair. -func DecodePrivateKey(keyType KeyType, privateKey []byte) (*KeyPair, error) { - switch keyType { - case RSA: - key, err := x509.ParsePKCS1PrivateKey(privateKey) - if err != nil { - k, e := x509.ParsePKCS8PrivateKey(privateKey) - return New(keyType, k), e - } - return New(keyType, key), nil - case ECDSA: - key, err := x509.ParseECPrivateKey(privateKey) - if err != nil { - k, e := x509.ParsePKCS8PrivateKey(privateKey) - return New(keyType, k), e - } - return New(ECDSA, key), nil - } - - return nil, fmt.Errorf("cannot decode the private key of %s", keyType) -} - -// EncodedPublicKey gets the encoded public key -func (k *KeyPair) EncodedPublicKey() ([]byte, error) { - switch k.keyType { - case RSA: - key, err := k.GetRsaPrivateKey() - if err != nil { - return nil, err - } - return x509.MarshalPKIXPublicKey(&key.PublicKey) - case ECDSA: - key, _ := k.GetEcdsaPrivateKey() - return x509.MarshalPKIXPublicKey(&key.PublicKey) - } - - return nil, fmt.Errorf("cannot decode the public key of %s", k.keyType) -} - -// DecodeRSAPublicKey parses the rsa public key. -func DecodeRSAPublicKey(publicKey []byte) (*rsa.PublicKey, error) { - pri, err := x509.ParsePKIXPublicKey(publicKey) - if err != nil { - p, e := x509.ParsePKCS1PublicKey(publicKey) - return p, e - } - return pri.(*rsa.PublicKey), err -} - -// DecodeECDSAPublicKey parses the ecdsa public key -func DecodeECDSAPublicKey(publicKey []byte) (*ecdsa.PublicKey, error) { - pubKey, err := x509.ParsePKIXPublicKey(publicKey) - if err != nil { - return nil, err - } - if ecdsaPubKey, ok := pubKey.(*ecdsa.PublicKey); ok { - return ecdsaPubKey, nil - } - return nil, errors.New("the public key is not generated using ECDSA signature algorithm") -} - -// GetRsaPrivateKey gets the rsa private key if you are using rsa signature -// algorithm to generate the private key -func (k *KeyPair) GetRsaPrivateKey() (*rsa.PrivateKey, error) { - if k.keyType != RSA { - return nil, errors.New("the private key is not generated using RSA signature algorithm") - } - if rsaKey, ok := k.privateKey.(*rsa.PrivateKey); ok { - return rsaKey, nil - } - return nil, errors.New("the private key is not generated using RSA signature algorithm") -} - -// GetEcdsaPrivateKey gets the ecdsa private key if you are using ecdsa signature -// algorithm to generate the private key -func (k *KeyPair) GetEcdsaPrivateKey() (*ecdsa.PrivateKey, error) { - if k.keyType != ECDSA { - return nil, errors.New("the private key is not generated using ECDSA signature algorithm") - } - if ecdsaKey, ok := k.privateKey.(*ecdsa.PrivateKey); ok { - return ecdsaKey, nil - } - return nil, errors.New("the private key is not generated using ecdsa signature algorithm") -} diff --git a/pulsaradmin/pkg/algorithm/rsa/rs256.go b/pulsaradmin/pkg/algorithm/rsa/rs256.go deleted file mode 100644 index 11fb320f31..0000000000 --- a/pulsaradmin/pkg/algorithm/rsa/rs256.go +++ /dev/null @@ -1,41 +0,0 @@ -// 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 rsa - -import ( - "crypto/rand" - "crypto/rsa" - - "github.com/pkg/errors" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" -) - -type RS256 struct{} - -func (p *RS256) GenerateSecret() ([]byte, error) { - return nil, errors.New("unsupported operation") -} - -func (p *RS256) GenerateKeyPair() (*keypair.KeyPair, error) { - pri, err := rsa.GenerateKey(rand.Reader, 2048) - if err != nil { - return nil, err - } - return keypair.New(keypair.RSA, pri), nil -} diff --git a/pulsaradmin/pkg/algorithm/rsa/rs384.go b/pulsaradmin/pkg/algorithm/rsa/rs384.go deleted file mode 100644 index ea6d984672..0000000000 --- a/pulsaradmin/pkg/algorithm/rsa/rs384.go +++ /dev/null @@ -1,41 +0,0 @@ -// 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 rsa - -import ( - "crypto/rand" - "crypto/rsa" - - "github.com/pkg/errors" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" -) - -type RS384 struct{} - -func (p *RS384) GenerateSecret() ([]byte, error) { - return nil, errors.New("unsupported operation") -} - -func (p *RS384) GenerateKeyPair() (*keypair.KeyPair, error) { - pri, err := rsa.GenerateKey(rand.Reader, 3072) - if err != nil { - return nil, err - } - return keypair.New(keypair.RSA, pri), nil -} diff --git a/pulsaradmin/pkg/algorithm/rsa/rs512.go b/pulsaradmin/pkg/algorithm/rsa/rs512.go deleted file mode 100644 index fbe7da0d2b..0000000000 --- a/pulsaradmin/pkg/algorithm/rsa/rs512.go +++ /dev/null @@ -1,41 +0,0 @@ -// 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 rsa - -import ( - "crypto/rand" - "crypto/rsa" - - "github.com/pkg/errors" - - "github.com/streamnative/pulsar-admin-go/pkg/algorithm/keypair" -) - -type RS512 struct{} - -func (p *RS512) GenerateSecret() ([]byte, error) { - return nil, errors.New("unsupported operation") -} - -func (p *RS512) GenerateKeyPair() (*keypair.KeyPair, error) { - pri, err := rsa.GenerateKey(rand.Reader, 4096) - if err != nil { - return nil, err - } - return keypair.New(keypair.RSA, pri), nil -} From 8234bb7856aa2de95db862ebe3d5c437827bf686 Mon Sep 17 00:00:00 2001 From: Max Xu Date: Thu, 16 Mar 2023 22:38:06 +0800 Subject: [PATCH 246/348] ci: add release workflow (streamnative/pulsar-admin-go#8) * ci: add release workflow * ci: add goreleaser config file --------- Signed-off-by: Max Xu --- pulsaradmin/.github/workflows/release.yaml | 48 ++++++++++++++++++++++ pulsaradmin/.goreleaser.yml | 4 ++ 2 files changed, 52 insertions(+) create mode 100644 pulsaradmin/.github/workflows/release.yaml create mode 100644 pulsaradmin/.goreleaser.yml diff --git a/pulsaradmin/.github/workflows/release.yaml b/pulsaradmin/.github/workflows/release.yaml new file mode 100644 index 0000000000..315507396f --- /dev/null +++ b/pulsaradmin/.github/workflows/release.yaml @@ -0,0 +1,48 @@ +# 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. +# + +name: Release + +on: + push: + tags: + - 'v[0-9]+.[0-9]+.[0-9]+-?*' + +permissions: + contents: write + +jobs: + release: + runs-on: ubuntu-latest + steps: + - name: Checkout + uses: actions/checkout@v3 + with: + fetch-depth: 0 + + - name: Setup Go + uses: actions/setup-go@v3 + + - name: Run GoReleaser + uses: goreleaser/goreleaser-action@v4 + with: + distribution: goreleaser + version: latest + args: release --clean + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} \ No newline at end of file diff --git a/pulsaradmin/.goreleaser.yml b/pulsaradmin/.goreleaser.yml new file mode 100644 index 0000000000..2986840045 --- /dev/null +++ b/pulsaradmin/.goreleaser.yml @@ -0,0 +1,4 @@ +builds: + - skip: true +release: + prerelease: auto From 87905b05e4a4829e83fbee1d2429379f05f0b26c Mon Sep 17 00:00:00 2001 From: Max Xu Date: Fri, 17 Mar 2023 23:15:28 +0800 Subject: [PATCH 247/348] docs: add README, LICENSE and CONTRIBUTING (streamnative/pulsar-admin-go#10) * docs: add LICENSE * docs: add contributing guides * docs: add README --------- Signed-off-by: Max Xu --- pulsaradmin/CONTRIBUTING.md | 76 ++++++++++++++ pulsaradmin/LICENSE | 202 ++++++++++++++++++++++++++++++++++++ pulsaradmin/README.md | 145 ++++++++++++++++++++++++++ 3 files changed, 423 insertions(+) create mode 100644 pulsaradmin/CONTRIBUTING.md create mode 100644 pulsaradmin/LICENSE create mode 100644 pulsaradmin/README.md diff --git a/pulsaradmin/CONTRIBUTING.md b/pulsaradmin/CONTRIBUTING.md new file mode 100644 index 0000000000..58ccfdfb69 --- /dev/null +++ b/pulsaradmin/CONTRIBUTING.md @@ -0,0 +1,76 @@ + + +# Contributing guidelines + +## Project structure +The overall project structure is illustrated below: + +```shell +├── pkg/ +│   ├── admin/ +│   │   ├── auth/ +│   │   ├── config/ +│   ├── rest/ +│   └── utils/ +├── alias.go +├── go.mod +└── go.sum +``` + +- The `alias.go` file in the root defines `pulsaradmin` package scope, which contains shortcuts of some types and functions from the `pkg`. +- The `pkg/admin` package contains all operations for pulsar admin resources. *Note: We should add a new file here if we wanna support a new resource.* + - The `pkg/admin/config` package contains configuration options for constructing a pulsar admin client. + - The `pkg/admin/auth` package contains auth providers which work in transport layer. +- The `pkg/rest` package contains a wrapped HTTP client for requesting pulsar REST API. +- The `pkg/utils` package contains common data structures and functions. + +## Contributing steps +1. Submit an issue describing your proposed change. +2. Discuss and wait for proposal to be accepted. +3. Fork this repo, develop and test your code changes. +4. Submit a pull request. + +## Conventions + +### Code conventions + +- [Go Code Review Comments](/~https://github.com/golang/go/wiki/CodeReviewComments) +- [Effective Go](https://golang.org/doc/effective_go.html) +- Know and avoid [Go landmines](https://gist.github.com/lavalamp/4bd23295a9f32706a48f) +- Commenting + - [Go's commenting conventions](http://blog.golang.org/godoc-documenting-go-code) + - If reviewers ask questions about why the code is the way it is, that's a sign that comments might be helpful. +- Naming + - Please consider package name when selecting an interface name, and avoid redundancy. For example, `storage.Interface` is better than `storage.StorageInterface`. + - Do not use uppercase characters, underscores, or dashes in package names. + - Please consider parent directory name when choosing a package name. For example, `pkg/controllers/autoscaler/foo.go` should say `package autoscaler` not `package autoscalercontroller`. + - Unless there's a good reason, the `package foo` line should match the name of the directory in which the `.go` file exists. + - Importers can use a different name if they need to disambiguate. + - Locks should be called `lock` and should never be embedded (always `lock sync.Mutex`). When multiple locks are present, give each lock a distinct name following Go conventions: `stateLock`, `mapLock` etc. + +### Folder and file conventions + +- All filenames should be lowercase. +- Go source files and directories use underscores, not dashes. + - Package directories should generally avoid using separators as much as possible. When package names are multiple words, they usually should be in nested subdirectories. +- Document directories and filenames should use dashes rather than underscores. +- All source files should add a license at the beginning. \ No newline at end of file diff --git a/pulsaradmin/LICENSE b/pulsaradmin/LICENSE new file mode 100644 index 0000000000..d645695673 --- /dev/null +++ b/pulsaradmin/LICENSE @@ -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/pulsaradmin/README.md b/pulsaradmin/README.md new file mode 100644 index 0000000000..7e3c8d0f33 --- /dev/null +++ b/pulsaradmin/README.md @@ -0,0 +1,145 @@ + + +# Pulsar Admin Go Library + +[![Language](https://img.shields.io/badge/Language-Go-blue.svg)](https://golang.org/) +[![License](https://img.shields.io/badge/License-Apache_2.0-blue.svg)](/~https://github.com/streamnative/pulsar-admin-go/blob/master/LICENSE) + +Pulsar-Admin-Go is a [Go](https://go.dev) library for [Apache Pulsar](https://pulsar.apache.org/). It provides a unified Go API for managing pulsar resources such as tenants, namespaces and topics, etc. + +## Motivation + +Currently many projects (e.g, [terraform-provider-pulsar](/~https://github.com/streamnative/terraform-provider-pulsar) and [pulsar-resources-operator](/~https://github.com/streamnative/pulsar-resources-operator)) +that need to manipulate the pulsar admin resources rely on the [pulsarctl](/~https://github.com/streamnative/pulsarctl), +which poses challenges for dependency management and versioning as we have to release a new pulsarctl to get updates. +So we decoupled the pulsar admin related api from pulsarctl and created the [pulsar-admin-go](/~https://github.com/streamnative/pulsar-admin-go) library based on it, +which also provides a clearer perspective and maintainability from an architectural perspective. + +## Quickstart + +### Prerequisite + +- go1.18+ +- pulsar-admin-go in go.mod + + ```shell + go get github.com/streamnative/pulsar-admin-go + ``` + +### Manage pulsar tenants + +- List all tenants + +```go +import ( + "github.com/streamnative/pulsar-admin-go" +) + +func main() { + cfg := &pulsaradmin.Config{} + admin, err := pulsaradmin.NewClient(cfg) + if err != nil { + panic(err) + } + + tenants, err := admin.Tenants().List() +} +``` + +### Manage pulsar namespaces + +- List all namespaces + +```go +import ( + "github.com/streamnative/pulsar-admin-go" +) + +func main() { + cfg := &pulsaradmin.Config{} + admin, err := pulsaradmin.NewClient(cfg) + if err != nil { + panic(err) + } + + namespaces, err := admin.Namespaces().GetNamespaces("public") +} +``` + +- Create a new namespace + +```go +import ( + "github.com/streamnative/pulsar-admin-go" +) + +func main() { + cfg := &pulsaradmin.Config{} + admin, err := pulsaradmin.NewClient(cfg) + if err != nil { + panic(err) + } + + if err := admin.Namespaces().CreateNamespace("public/dev"); err != nil { + panic(err) + } +} +``` + +### Manage pulsar topics + +- Create a topic + +```go + +import ( + "github.com/streamnative/pulsar-admin-go" + "github.com/streamnative/pulsar-admin-go/pkg/utils" +) + +func main() { + cfg := &pulsaradmin.Config{} + admin, err := pulsaradmin.NewClient(cfg) + if err != nil { + panic(err) + } + + topic, err := utils.GetTopicName("public/dev/topic") + if err != nil { + panic(err) + } + + if err := admin.Topics().Create(*topic, 3); err != nil { + panic(err) + } +} +``` + +## Contributing + +Contributions are warmly welcomed and greatly appreciated! +The project follows the typical GitHub pull request model. See [CONTRIBUTING.md](CONTRIBUTING.md) for more details. +Before starting any work, please either comment on an existing issue, or file a new one. + +## License + +Licensed under the [Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0). See [LICENSE](LICENSE) \ No newline at end of file From 9edc7fdd1e5fba864fbfd33f11a86da189d98367 Mon Sep 17 00:00:00 2001 From: Eric Shen Date: Sat, 18 Mar 2023 18:57:57 +0800 Subject: [PATCH 248/348] fix: change copyright from ASF to StreamNative (streamnative/pulsar-admin-go#11) * fix: change copyright from ASF to StreamNative --------- Signed-off-by: ericsyh --- pulsaradmin/.github/CODEOWNERS | 16 ++++++++++++++++ pulsaradmin/.github/workflows/ci.yaml | 12 +++++------- pulsaradmin/.github/workflows/release.yaml | 12 +++++------- pulsaradmin/.gitignore | 16 ++++++++++++++++ pulsaradmin/.golangci.yaml | 12 +++++------- pulsaradmin/.goreleaser.yml | 16 ++++++++++++++++ pulsaradmin/CONTRIBUTING.md | 10 ++++------ pulsaradmin/README.md | 10 ++++------ pulsaradmin/alias.go | 12 +++++------- pulsaradmin/pkg/admin/admin.go | 12 +++++------- pulsaradmin/pkg/admin/admin_test.go | 12 +++++------- pulsaradmin/pkg/admin/auth/oauth2.go | 12 +++++------- pulsaradmin/pkg/admin/auth/oauth2_test.go | 12 +++++------- pulsaradmin/pkg/admin/auth/provider.go | 12 +++++------- pulsaradmin/pkg/admin/auth/tls.go | 12 +++++------- pulsaradmin/pkg/admin/auth/token.go | 12 +++++------- pulsaradmin/pkg/admin/auth/transport.go | 12 +++++------- pulsaradmin/pkg/admin/broker_stats.go | 12 +++++------- pulsaradmin/pkg/admin/brokers.go | 12 +++++------- pulsaradmin/pkg/admin/cluster.go | 12 +++++------- pulsaradmin/pkg/admin/config/api_version.go | 12 +++++------- pulsaradmin/pkg/admin/config/api_version_test.go | 12 +++++------- pulsaradmin/pkg/admin/config/config.go | 12 +++++------- pulsaradmin/pkg/admin/functions.go | 12 +++++------- pulsaradmin/pkg/admin/functions_worker.go | 12 +++++------- pulsaradmin/pkg/admin/namespace.go | 12 +++++------- pulsaradmin/pkg/admin/ns_isolation_policy.go | 12 +++++------- pulsaradmin/pkg/admin/packages.go | 12 +++++------- pulsaradmin/pkg/admin/resource_quotas.go | 12 +++++------- pulsaradmin/pkg/admin/schema.go | 12 +++++------- pulsaradmin/pkg/admin/sinks.go | 12 +++++------- pulsaradmin/pkg/admin/sources.go | 12 +++++------- pulsaradmin/pkg/admin/subscription.go | 12 +++++------- pulsaradmin/pkg/admin/tenant.go | 12 +++++------- pulsaradmin/pkg/admin/topic.go | 12 +++++------- pulsaradmin/pkg/rest/client.go | 12 +++++------- pulsaradmin/pkg/rest/client_test.go | 12 +++++------- pulsaradmin/pkg/rest/errors.go | 12 +++++------- pulsaradmin/pkg/utils/allocator_stats.go | 12 +++++------- pulsaradmin/pkg/utils/auth_action.go | 12 +++++------- pulsaradmin/pkg/utils/auth_polices.go | 12 +++++------- pulsaradmin/pkg/utils/auth_polices_test.go | 12 +++++------- pulsaradmin/pkg/utils/backlog_quota.go | 12 +++++------- .../pkg/utils/broker_ns_isolation_data.go | 12 +++++------- pulsaradmin/pkg/utils/bundles_data.go | 12 +++++------- pulsaradmin/pkg/utils/connector_definition.go | 12 +++++------- pulsaradmin/pkg/utils/consumer_config.go | 12 +++++------- pulsaradmin/pkg/utils/data.go | 12 +++++------- pulsaradmin/pkg/utils/dispatch_rate.go | 12 +++++------- pulsaradmin/pkg/utils/function_confg.go | 12 +++++------- pulsaradmin/pkg/utils/function_state.go | 12 +++++------- pulsaradmin/pkg/utils/function_status.go | 12 +++++------- pulsaradmin/pkg/utils/functions_stats.go | 12 +++++------- pulsaradmin/pkg/utils/home_dir.go | 12 +++++------- pulsaradmin/pkg/utils/inactive_topic_policies.go | 12 +++++------- .../pkg/utils/internal_configuration_data.go | 12 +++++------- pulsaradmin/pkg/utils/load_manage_report.go | 12 +++++------- .../pkg/utils/long_running_process_status.go | 12 +++++------- pulsaradmin/pkg/utils/message.go | 12 +++++------- pulsaradmin/pkg/utils/message_id.go | 12 +++++------- pulsaradmin/pkg/utils/message_id_test.go | 12 +++++------- pulsaradmin/pkg/utils/metrics.go | 12 +++++------- pulsaradmin/pkg/utils/namespace_name.go | 12 +++++------- pulsaradmin/pkg/utils/namespace_name_test.go | 12 +++++------- pulsaradmin/pkg/utils/ns_isolation_data.go | 12 +++++------- pulsaradmin/pkg/utils/ns_ownership_status.go | 12 +++++------- pulsaradmin/pkg/utils/package_metadata.go | 12 +++++------- pulsaradmin/pkg/utils/package_name.go | 12 +++++------- pulsaradmin/pkg/utils/package_name_test.go | 12 +++++------- pulsaradmin/pkg/utils/package_type.go | 12 +++++------- pulsaradmin/pkg/utils/persistence_policies.go | 12 +++++------- pulsaradmin/pkg/utils/policies.go | 12 +++++------- pulsaradmin/pkg/utils/publish_rate.go | 12 +++++------- pulsaradmin/pkg/utils/resource_quota.go | 12 +++++------- pulsaradmin/pkg/utils/resources.go | 12 +++++------- pulsaradmin/pkg/utils/retention_policies.go | 12 +++++------- pulsaradmin/pkg/utils/schema_strategy.go | 12 +++++------- pulsaradmin/pkg/utils/schema_util.go | 12 +++++------- pulsaradmin/pkg/utils/sink_config.go | 12 +++++------- pulsaradmin/pkg/utils/sink_status.go | 12 +++++------- pulsaradmin/pkg/utils/source_config.go | 12 +++++------- pulsaradmin/pkg/utils/source_status.go | 12 +++++------- pulsaradmin/pkg/utils/subscription_auth_mode.go | 12 +++++------- .../pkg/utils/topic_auto_creation_config.go | 12 +++++------- pulsaradmin/pkg/utils/topic_domain.go | 12 +++++------- pulsaradmin/pkg/utils/topic_name.go | 12 +++++------- pulsaradmin/pkg/utils/topic_name_test.go | 12 +++++------- pulsaradmin/pkg/utils/topic_type.go | 12 +++++------- pulsaradmin/pkg/utils/topics_stats_stream.go | 12 +++++------- pulsaradmin/pkg/utils/update_options.go | 12 +++++------- pulsaradmin/pkg/utils/utils.go | 12 +++++------- pulsaradmin/pkg/utils/utils_test.go | 14 ++++++-------- pulsaradmin/pkg/utils/window_confing.go | 12 +++++------- pulsaradmin/pkg/utils/worker_info.go | 12 +++++------- 94 files changed, 502 insertions(+), 636 deletions(-) diff --git a/pulsaradmin/.github/CODEOWNERS b/pulsaradmin/.github/CODEOWNERS index c3875a1248..b22c6f1824 100644 --- a/pulsaradmin/.github/CODEOWNERS +++ b/pulsaradmin/.github/CODEOWNERS @@ -1,2 +1,18 @@ +# Copyright 2023 StreamNative, Inc. +# +# 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. +# + # Owning team's slack is #f_sn_data_plane * @streamnative/cloud diff --git a/pulsaradmin/.github/workflows/ci.yaml b/pulsaradmin/.github/workflows/ci.yaml index 0fc45a3ba6..69daea3772 100644 --- a/pulsaradmin/.github/workflows/ci.yaml +++ b/pulsaradmin/.github/workflows/ci.yaml @@ -1,10 +1,8 @@ -# 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 +# Copyright 2023 StreamNative, Inc. +# +# 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 # diff --git a/pulsaradmin/.github/workflows/release.yaml b/pulsaradmin/.github/workflows/release.yaml index 315507396f..0f91e2468b 100644 --- a/pulsaradmin/.github/workflows/release.yaml +++ b/pulsaradmin/.github/workflows/release.yaml @@ -1,10 +1,8 @@ -# 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 +# Copyright 2023 StreamNative, Inc. +# +# 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 # diff --git a/pulsaradmin/.gitignore b/pulsaradmin/.gitignore index 76a786dfd8..846135c664 100644 --- a/pulsaradmin/.gitignore +++ b/pulsaradmin/.gitignore @@ -1,3 +1,19 @@ +# Copyright 2023 StreamNative, Inc. +# +# 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. +# + .DS_Store .idea/ .vscode/ diff --git a/pulsaradmin/.golangci.yaml b/pulsaradmin/.golangci.yaml index 292947d170..8f897e871f 100644 --- a/pulsaradmin/.golangci.yaml +++ b/pulsaradmin/.golangci.yaml @@ -1,11 +1,9 @@ # -# 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 +# Copyright 2023 StreamNative, Inc. +# +# 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 # diff --git a/pulsaradmin/.goreleaser.yml b/pulsaradmin/.goreleaser.yml index 2986840045..513e957378 100644 --- a/pulsaradmin/.goreleaser.yml +++ b/pulsaradmin/.goreleaser.yml @@ -1,3 +1,19 @@ +# Copyright 2023 StreamNative, Inc. +# +# 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. +# + builds: - skip: true release: diff --git a/pulsaradmin/CONTRIBUTING.md b/pulsaradmin/CONTRIBUTING.md index 58ccfdfb69..8a8d4d49d8 100644 --- a/pulsaradmin/CONTRIBUTING.md +++ b/pulsaradmin/CONTRIBUTING.md @@ -1,11 +1,9 @@ + + + + +Fixes #TODO + +## Motivation + + + +## Modifications + + + +## Verification + + diff --git a/pulsaradmin/.licenserc.yaml b/pulsaradmin/.licenserc.yaml index 6f4cee6655..fd8411e049 100644 --- a/pulsaradmin/.licenserc.yaml +++ b/pulsaradmin/.licenserc.yaml @@ -24,5 +24,6 @@ header: - '**/go.sum' - 'LICENSE' - 'NOTICE' + - PULL_REQUEST_TEMPLATE.md comment: on-failure From 1be51339b6b7735ac70ee8ea06e5aac260989304 Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Wed, 19 Jul 2023 10:06:55 +0800 Subject: [PATCH 265/348] feat: create parent dir for package downloading when necessary (streamnative/pulsar-admin-go#35) Create parent dir for package downloading when necessary --- pulsaradmin/pkg/admin/packages.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/pulsaradmin/pkg/admin/packages.go b/pulsaradmin/pkg/admin/packages.go index 8b56645040..67544b5eeb 100644 --- a/pulsaradmin/pkg/admin/packages.go +++ b/pulsaradmin/pkg/admin/packages.go @@ -23,6 +23,7 @@ import ( "mime/multipart" "net/textproto" "os" + "path" "path/filepath" "strings" @@ -96,6 +97,15 @@ func (p packages) Download(packageURL, destinationFile string) error { } endpoint := p.pulsar.endpoint(p.basePath, string(packageName.GetType()), packageName.GetTenant(), packageName.GetNamespace(), packageName.GetName(), packageName.GetVersion()) + + parent := path.Dir(destinationFile) + if parent != "." { + err = os.MkdirAll(parent, 0755) + if err != nil { + return fmt.Errorf("failed to create parent directory %s: %w", parent, err) + } + } + _, err = os.Open(destinationFile) if err != nil { if !os.IsNotExist(err) { From dd5b83b39d82d6284ecf4136bd0446245d7065ef Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Wed, 19 Jul 2023 10:48:52 +0800 Subject: [PATCH 266/348] fix: use pointer for struct field so the default value from unmarshal is nil (streamnative/pulsar-admin-go#36) Use pointer for struct field so the default value from unmarshal is nil Co-authored-by: Neng Lu --- pulsaradmin/pkg/utils/consumer_config.go | 2 +- pulsaradmin/pkg/utils/data.go | 1 + pulsaradmin/pkg/utils/function_confg.go | 2 +- pulsaradmin/pkg/utils/producer_config.go | 8 ++++---- pulsaradmin/pkg/utils/source_config.go | 6 +++--- 5 files changed, 10 insertions(+), 9 deletions(-) diff --git a/pulsaradmin/pkg/utils/consumer_config.go b/pulsaradmin/pkg/utils/consumer_config.go index 5ffe1abd04..6b0dec91a0 100644 --- a/pulsaradmin/pkg/utils/consumer_config.go +++ b/pulsaradmin/pkg/utils/consumer_config.go @@ -22,6 +22,6 @@ type ConsumerConfig struct { ReceiverQueueSize int `json:"receiverQueueSize,omitempty" yaml:"receiverQueueSize"` SchemaProperties map[string]string `json:"schemaProperties,omitempty" yaml:"schemaProperties"` ConsumerProperties map[string]string `json:"consumerProperties,omitempty" yaml:"consumerProperties"` - CryptoConfig CryptoConfig `json:"cryptoConfig,omitempty" yaml:"cryptoConfig"` + CryptoConfig *CryptoConfig `json:"cryptoConfig,omitempty" yaml:"cryptoConfig"` PoolMessages bool `json:"poolMessages,omitempty" yaml:"poolMessages"` } diff --git a/pulsaradmin/pkg/utils/data.go b/pulsaradmin/pkg/utils/data.go index 8ae3d71fd0..ea9b9046a6 100644 --- a/pulsaradmin/pkg/utils/data.go +++ b/pulsaradmin/pkg/utils/data.go @@ -163,6 +163,7 @@ type SinkData struct { MaxMessageRetries int `json:"maxMessageRetries,omitempty"` DeadLetterTopic string `json:"deadLetterTopic,omitempty"` ProcessingGuarantees string `json:"processingGuarantees,omitempty"` + RetainKeyOrdering bool `json:"retainKeyOrdering,omitempty"` Archive string `json:"archive,omitempty"` ClassName string `json:"className,omitempty"` SinkConfigFile string `json:"sinkConfigFile,omitempty"` diff --git a/pulsaradmin/pkg/utils/function_confg.go b/pulsaradmin/pkg/utils/function_confg.go index 2e365f425c..0f9bc56bab 100644 --- a/pulsaradmin/pkg/utils/function_confg.go +++ b/pulsaradmin/pkg/utils/function_confg.go @@ -36,7 +36,7 @@ type FunctionConfig struct { Output string `json:"output,omitempty" yaml:"output"` - ProducerConfig ProducerConfig `json:"producerConfig,omitempty" yaml:"producerConfig"` + ProducerConfig *ProducerConfig `json:"producerConfig,omitempty" yaml:"producerConfig"` CustomSchemaOutputs map[string]string `json:"customSchemaOutputs,omitempty" yaml:"customSchemaOutputs"` OutputSerdeClassName string `json:"outputSerdeClassName,omitempty" yaml:"outputSerdeClassName"` diff --git a/pulsaradmin/pkg/utils/producer_config.go b/pulsaradmin/pkg/utils/producer_config.go index 3b4c52a271..591fa01257 100644 --- a/pulsaradmin/pkg/utils/producer_config.go +++ b/pulsaradmin/pkg/utils/producer_config.go @@ -20,8 +20,8 @@ type ProducerConfig struct { //nolint MaxPendingMessagesAcrossPartitions int `json:"maxPendingMessagesAcrossPartitions" yaml:"maxPendingMessagesAcrossPartitions"` - UseThreadLocalProducers bool `json:"useThreadLocalProducers" yaml:"useThreadLocalProducers"` - CryptoConfig CryptoConfig `json:"cryptoConfig" yaml:"cryptoConfig"` - BatchBuilder string `json:"batchBuilder" yaml:"batchBuilder"` - CompressionType string `json:"compressionType" yaml:"compressionType"` + UseThreadLocalProducers bool `json:"useThreadLocalProducers" yaml:"useThreadLocalProducers"` + CryptoConfig *CryptoConfig `json:"cryptoConfig" yaml:"cryptoConfig"` + BatchBuilder string `json:"batchBuilder" yaml:"batchBuilder"` + CompressionType string `json:"compressionType" yaml:"compressionType"` } diff --git a/pulsaradmin/pkg/utils/source_config.go b/pulsaradmin/pkg/utils/source_config.go index bfe031c3d3..506d3eb260 100644 --- a/pulsaradmin/pkg/utils/source_config.go +++ b/pulsaradmin/pkg/utils/source_config.go @@ -21,7 +21,7 @@ type SourceConfig struct { Name string `json:"name,omitempty" yaml:"name"` ClassName string `json:"className,omitempty" yaml:"className"` - ProducerConfig ProducerConfig `json:"producerConfig,omitempty" yaml:"producerConfig"` + ProducerConfig *ProducerConfig `json:"producerConfig,omitempty" yaml:"producerConfig"` TopicName string `json:"topicName,omitempty" yaml:"topicName"` SerdeClassName string `json:"serdeClassName,omitempty" yaml:"serdeClassName"` @@ -45,6 +45,6 @@ type SourceConfig struct { CustomRuntimeOptions string `json:"customRuntimeOptions,omitempty" yaml:"customRuntimeOptions"` - BatchSourceConfig BatchSourceConfig `json:"batchSourceConfig,omitempty" yaml:"batchSourceConfig"` - BatchBuilder string `json:"batchBuilder,omitempty" yaml:"batchBuilder"` + BatchSourceConfig *BatchSourceConfig `json:"batchSourceConfig,omitempty" yaml:"batchSourceConfig"` + BatchBuilder string `json:"batchBuilder,omitempty" yaml:"batchBuilder"` } From de96e1744fa414b79587a4aabbd3d74aa20918b1 Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Fri, 21 Jul 2023 06:09:08 +0800 Subject: [PATCH 267/348] fix: remove omitempty for bool value (streamnative/pulsar-admin-go#38) --- pulsaradmin/pkg/utils/consumer_config.go | 2 +- pulsaradmin/pkg/utils/function_confg.go | 14 +++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/pulsaradmin/pkg/utils/consumer_config.go b/pulsaradmin/pkg/utils/consumer_config.go index 6b0dec91a0..2502edf6df 100644 --- a/pulsaradmin/pkg/utils/consumer_config.go +++ b/pulsaradmin/pkg/utils/consumer_config.go @@ -18,7 +18,7 @@ package utils type ConsumerConfig struct { SchemaType string `json:"schemaType,omitempty" yaml:"schemaType"` SerdeClassName string `json:"serdeClassName,omitempty" yaml:"serdeClassName"` - IsRegexPattern bool `json:"isRegexPattern,omitempty" yaml:"isRegexPattern"` + RegexPattern bool `json:"regexPattern,omitempty" yaml:"regexPattern"` ReceiverQueueSize int `json:"receiverQueueSize,omitempty" yaml:"receiverQueueSize"` SchemaProperties map[string]string `json:"schemaProperties,omitempty" yaml:"schemaProperties"` ConsumerProperties map[string]string `json:"consumerProperties,omitempty" yaml:"consumerProperties"` diff --git a/pulsaradmin/pkg/utils/function_confg.go b/pulsaradmin/pkg/utils/function_confg.go index 0f9bc56bab..ce7017e362 100644 --- a/pulsaradmin/pkg/utils/function_confg.go +++ b/pulsaradmin/pkg/utils/function_confg.go @@ -25,12 +25,12 @@ type FunctionConfig struct { TimeoutMs *int64 `json:"timeoutMs,omitempty" yaml:"timeoutMs"` TopicsPattern *string `json:"topicsPattern,omitempty" yaml:"topicsPattern"` // Whether the subscriptions the functions created/used should be deleted when the functions is deleted - CleanupSubscription bool `json:"cleanupSubscription,omitempty" yaml:"cleanupSubscription"` - RetainOrdering bool `json:"retainOrdering,omitempty" yaml:"retainOrdering"` - RetainKeyOrdering bool `json:"retainKeyOrdering,omitempty" yaml:"retainKeyOrdering"` + CleanupSubscription bool `json:"cleanupSubscription" yaml:"cleanupSubscription"` + RetainOrdering bool `json:"retainOrdering" yaml:"retainOrdering"` + RetainKeyOrdering bool `json:"retainKeyOrdering" yaml:"retainKeyOrdering"` BatchBuilder string `json:"batchBuilder,omitempty" yaml:"batchBuilder"` - ForwardSourceMessageProperty bool `json:"forwardSourceMessageProperty,omitempty" yaml:"forwardSourceMessageProperty"` - AutoAck bool `json:"autoAck,omitempty" yaml:"autoAck"` + ForwardSourceMessageProperty bool `json:"forwardSourceMessageProperty" yaml:"forwardSourceMessageProperty"` + AutoAck bool `json:"autoAck" yaml:"autoAck"` Parallelism int `json:"parallelism,omitempty" yaml:"parallelism"` MaxMessageRetries *int `json:"maxMessageRetries,omitempty" yaml:"maxMessageRetries"` @@ -86,7 +86,7 @@ type FunctionConfig struct { MaxPendingAsyncRequests int `json:"maxPendingAsyncRequests,omitempty" yaml:"maxPendingAsyncRequests"` //nolint - ExposePulsarAdminClientEnabled bool `json:"exposePulsarAdminClientEnabled,omitempty" yaml:"exposePulsarAdminClientEnabled"` - SkipToLatest bool `json:"skipToLatest,omitempty" yaml:"skipToLatest"` + ExposePulsarAdminClientEnabled bool `json:"exposePulsarAdminClientEnabled" yaml:"exposePulsarAdminClientEnabled"` + SkipToLatest bool `json:"skipToLatest" yaml:"skipToLatest"` SubscriptionPosition string `json:"subscriptionPosition,omitempty" yaml:"subscriptionPosition"` } From eee6d76eea7c6524e438f8a22aa36d231e203c65 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Sun, 13 Aug 2023 14:15:29 +0800 Subject: [PATCH 268/348] fix: Fix wrong model field (streamnative/pulsar-admin-go#42) Fix wrong model field --- pulsaradmin/pkg/utils/message_id.go | 12 ++++++------ pulsaradmin/pkg/utils/message_id_test.go | 6 +++--- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/pulsaradmin/pkg/utils/message_id.go b/pulsaradmin/pkg/utils/message_id.go index 87d8e4a17f..014de3a050 100644 --- a/pulsaradmin/pkg/utils/message_id.go +++ b/pulsaradmin/pkg/utils/message_id.go @@ -23,10 +23,10 @@ import ( ) type MessageID struct { - LedgerID int64 `json:"ledgerId"` - EntryID int64 `json:"entryId"` - PartitionedIndex int `json:"partitionedIndex"` - BatchIndex int `json:"-"` + LedgerID int64 `json:"ledgerId"` + EntryID int64 `json:"entryId"` + PartitionIndex int `json:"partitionIndex"` + BatchIndex int `json:"-"` } var Latest = MessageID{0x7fffffffffffffff, 0x7fffffffffffffff, -1, -1} @@ -58,7 +58,7 @@ func ParseMessageID(str string) (*MessageID, error) { if err != nil { return nil, errors.Errorf("invalid partition index. %s", str) } - m.PartitionedIndex = pi + m.PartitionIndex = pi } if len(s) == 4 { @@ -75,6 +75,6 @@ func ParseMessageID(str string) (*MessageID, error) { func (m MessageID) String() string { return strconv.FormatInt(m.LedgerID, 10) + ":" + strconv.FormatInt(m.EntryID, 10) + ":" + - strconv.Itoa(m.PartitionedIndex) + ":" + + strconv.Itoa(m.PartitionIndex) + ":" + strconv.Itoa(m.BatchIndex) } diff --git a/pulsaradmin/pkg/utils/message_id_test.go b/pulsaradmin/pkg/utils/message_id_test.go index 7d34cf7303..9c74befaaa 100644 --- a/pulsaradmin/pkg/utils/message_id_test.go +++ b/pulsaradmin/pkg/utils/message_id_test.go @@ -24,15 +24,15 @@ import ( func TestParseMessageId(t *testing.T) { id, err := ParseMessageID("1:1") assert.Nil(t, err) - assert.Equal(t, MessageID{LedgerID: 1, EntryID: 1, PartitionedIndex: -1, BatchIndex: -1}, *id) + assert.Equal(t, MessageID{LedgerID: 1, EntryID: 1, PartitionIndex: -1, BatchIndex: -1}, *id) id, err = ParseMessageID("1:2:3") assert.Nil(t, err) - assert.Equal(t, MessageID{LedgerID: 1, EntryID: 2, PartitionedIndex: 3, BatchIndex: -1}, *id) + assert.Equal(t, MessageID{LedgerID: 1, EntryID: 2, PartitionIndex: 3, BatchIndex: -1}, *id) id, err = ParseMessageID("1:2:3:4") assert.Nil(t, err) - assert.Equal(t, MessageID{LedgerID: 1, EntryID: 2, PartitionedIndex: 3, BatchIndex: 4}, *id) + assert.Equal(t, MessageID{LedgerID: 1, EntryID: 2, PartitionIndex: 3, BatchIndex: 4}, *id) } func TestParseMessageIdErrors(t *testing.T) { From 750e8e9d68a8d0c78a9a8bcd97363f60e57a8a6b Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 16 Aug 2023 00:07:55 +0800 Subject: [PATCH 269/348] chore: tidy pulsar-admin-go sources Signed-off-by: tison --- Makefile | 1 + go.mod | 5 +- go.sum | 1 - pulsaradmin/.github/CODEOWNERS | 18 - pulsaradmin/.github/PULL_REQUEST_TEMPLATE.md | 26 - pulsaradmin/.github/workflows/ci-pr.yaml | 35 - pulsaradmin/.github/workflows/ci.yaml | 65 -- pulsaradmin/.github/workflows/release.yaml | 46 -- pulsaradmin/.gitignore | 19 - pulsaradmin/.golangci.yaml | 40 -- pulsaradmin/.goreleaser.yml | 20 - pulsaradmin/.licenserc.yaml | 29 - pulsaradmin/CONTRIBUTING.md | 48 +- pulsaradmin/LICENSE | 202 ------ pulsaradmin/Makefile | 53 -- pulsaradmin/README.md | 58 +- pulsaradmin/alias.go | 4 +- pulsaradmin/go.mod | 32 - pulsaradmin/go.sum | 682 ------------------- pulsaradmin/pkg/admin/admin.go | 8 +- pulsaradmin/pkg/admin/admin_test.go | 4 +- pulsaradmin/pkg/admin/auth/provider.go | 2 +- pulsaradmin/pkg/admin/auth/transport.go | 2 +- pulsaradmin/pkg/admin/broker_stats.go | 2 +- pulsaradmin/pkg/admin/brokers.go | 2 +- pulsaradmin/pkg/admin/cluster.go | 2 +- pulsaradmin/pkg/admin/functions.go | 2 +- pulsaradmin/pkg/admin/functions_worker.go | 2 +- pulsaradmin/pkg/admin/namespace.go | 2 +- pulsaradmin/pkg/admin/ns_isolation_policy.go | 2 +- pulsaradmin/pkg/admin/packages.go | 2 +- pulsaradmin/pkg/admin/resource_quotas.go | 2 +- pulsaradmin/pkg/admin/schema.go | 2 +- pulsaradmin/pkg/admin/sinks.go | 2 +- pulsaradmin/pkg/admin/sources.go | 2 +- pulsaradmin/pkg/admin/subscription.go | 2 +- pulsaradmin/pkg/admin/tenant.go | 2 +- pulsaradmin/pkg/admin/topic.go | 2 +- 38 files changed, 60 insertions(+), 1370 deletions(-) delete mode 100644 pulsaradmin/.github/CODEOWNERS delete mode 100644 pulsaradmin/.github/PULL_REQUEST_TEMPLATE.md delete mode 100644 pulsaradmin/.github/workflows/ci-pr.yaml delete mode 100644 pulsaradmin/.github/workflows/ci.yaml delete mode 100644 pulsaradmin/.github/workflows/release.yaml delete mode 100644 pulsaradmin/.gitignore delete mode 100644 pulsaradmin/.golangci.yaml delete mode 100644 pulsaradmin/.goreleaser.yml delete mode 100644 pulsaradmin/.licenserc.yaml delete mode 100644 pulsaradmin/LICENSE delete mode 100644 pulsaradmin/Makefile delete mode 100644 pulsaradmin/go.mod delete mode 100644 pulsaradmin/go.sum diff --git a/Makefile b/Makefile index 433db8cb77..4eb590b06b 100644 --- a/Makefile +++ b/Makefile @@ -29,6 +29,7 @@ GOROOT ?= $(shell go env GOROOT) build: go build ./pulsar + go build ./pulsaradmin go build -o bin/pulsar-perf ./perf lint: bin/golangci-lint diff --git a/go.mod b/go.mod index 69862b2080..f88d6ad576 100644 --- a/go.mod +++ b/go.mod @@ -27,9 +27,11 @@ require ( golang.org/x/mod v0.5.1 golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 golang.org/x/time v0.0.0-20191024005414-555d28b269f0 - google.golang.org/protobuf v1.30.0 // indirect + google.golang.org/protobuf v1.30.0 ) +require github.com/golang/protobuf v1.5.2 + require ( github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect github.com/ardielle/ardielle-go v1.5.2 // indirect @@ -39,7 +41,6 @@ require ( github.com/dvsekhvalnov/jose2go v1.5.0 // indirect github.com/fsnotify/fsnotify v1.4.9 // indirect github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect - github.com/golang/protobuf v1.5.2 // indirect github.com/golang/snappy v0.0.1 // indirect github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/inconshreveable/mousetrap v1.0.1 // indirect diff --git a/go.sum b/go.sum index 3a3987443d..00a44917d8 100644 --- a/go.sum +++ b/go.sum @@ -551,7 +551,6 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0 h1:bxAC2xTBsZGibn2RTntX0oH50xLsqy1OxA9tTL3p/lk= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= diff --git a/pulsaradmin/.github/CODEOWNERS b/pulsaradmin/.github/CODEOWNERS deleted file mode 100644 index b22c6f1824..0000000000 --- a/pulsaradmin/.github/CODEOWNERS +++ /dev/null @@ -1,18 +0,0 @@ -# Copyright 2023 StreamNative, Inc. -# -# 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. -# - -# Owning team's slack is #f_sn_data_plane -* @streamnative/cloud diff --git a/pulsaradmin/.github/PULL_REQUEST_TEMPLATE.md b/pulsaradmin/.github/PULL_REQUEST_TEMPLATE.md deleted file mode 100644 index 50c245f674..0000000000 --- a/pulsaradmin/.github/PULL_REQUEST_TEMPLATE.md +++ /dev/null @@ -1,26 +0,0 @@ - - - - - -Fixes #TODO - -## Motivation - - - -## Modifications - - - -## Verification - - diff --git a/pulsaradmin/.github/workflows/ci-pr.yaml b/pulsaradmin/.github/workflows/ci-pr.yaml deleted file mode 100644 index a33769d781..0000000000 --- a/pulsaradmin/.github/workflows/ci-pr.yaml +++ /dev/null @@ -1,35 +0,0 @@ -# Copyright 2023 StreamNative, Inc. -# -# 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. -# - -name: CI / PR - -on: - pull_request: - branches: - - '*' - types: - - opened - - reopened - - edited - - synchronize - -jobs: - title-check: - runs-on: ubuntu-latest - steps: - - uses: amannn/action-semantic-pull-request@v5 - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} diff --git a/pulsaradmin/.github/workflows/ci.yaml b/pulsaradmin/.github/workflows/ci.yaml deleted file mode 100644 index 0390ac62c9..0000000000 --- a/pulsaradmin/.github/workflows/ci.yaml +++ /dev/null @@ -1,65 +0,0 @@ -# Copyright 2023 StreamNative, Inc. -# -# 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. -# - -name: CI - -on: - pull_request: - branches: - - '*' - -jobs: - license-check: - runs-on: ubuntu-latest - steps: - - name: Checkout - uses: actions/checkout@v3 - - - name: Check license header - uses: apache/skywalking-eyes@e1a02359b239bd28de3f6d35fdc870250fa513d5 - - lint: - runs-on: ubuntu-latest - steps: - - name: Checkout - uses: actions/checkout@v3 - - - name: Setup Go - uses: actions/setup-go@v3 - with: - go-version: 1.18 - - - name: Run golangci-lint - uses: golangci/golangci-lint-action@v3 - with: - version: v1.51.2 - - test: - runs-on: ubuntu-latest - steps: - - name: Checkout - uses: actions/checkout@v3 - - - name: Setup Go - uses: actions/setup-go@v3 - with: - go-version: 1.18 - - - name: Run build and test - run: | - go build ./... - go test -race ./... - diff --git a/pulsaradmin/.github/workflows/release.yaml b/pulsaradmin/.github/workflows/release.yaml deleted file mode 100644 index 0f91e2468b..0000000000 --- a/pulsaradmin/.github/workflows/release.yaml +++ /dev/null @@ -1,46 +0,0 @@ -# Copyright 2023 StreamNative, Inc. -# -# 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. -# - -name: Release - -on: - push: - tags: - - 'v[0-9]+.[0-9]+.[0-9]+-?*' - -permissions: - contents: write - -jobs: - release: - runs-on: ubuntu-latest - steps: - - name: Checkout - uses: actions/checkout@v3 - with: - fetch-depth: 0 - - - name: Setup Go - uses: actions/setup-go@v3 - - - name: Run GoReleaser - uses: goreleaser/goreleaser-action@v4 - with: - distribution: goreleaser - version: latest - args: release --clean - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} \ No newline at end of file diff --git a/pulsaradmin/.gitignore b/pulsaradmin/.gitignore deleted file mode 100644 index 846135c664..0000000000 --- a/pulsaradmin/.gitignore +++ /dev/null @@ -1,19 +0,0 @@ -# Copyright 2023 StreamNative, Inc. -# -# 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. -# - -.DS_Store -.idea/ -.vscode/ diff --git a/pulsaradmin/.golangci.yaml b/pulsaradmin/.golangci.yaml deleted file mode 100644 index 8f897e871f..0000000000 --- a/pulsaradmin/.golangci.yaml +++ /dev/null @@ -1,40 +0,0 @@ -# -# Copyright 2023 StreamNative, Inc. -# -# 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. -# - -run: - deadline: 6m - -linters: - # disable all for explicit enable - disable-all: true - enable: - - errcheck - - gosimple - - govet - - ineffassign - - staticcheck - - typecheck - - unused - - lll - - goimports - - bodyclose - - misspell - - prealloc - - revive - - stylecheck - - unconvert - - unparam diff --git a/pulsaradmin/.goreleaser.yml b/pulsaradmin/.goreleaser.yml deleted file mode 100644 index 513e957378..0000000000 --- a/pulsaradmin/.goreleaser.yml +++ /dev/null @@ -1,20 +0,0 @@ -# Copyright 2023 StreamNative, Inc. -# -# 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. -# - -builds: - - skip: true -release: - prerelease: auto diff --git a/pulsaradmin/.licenserc.yaml b/pulsaradmin/.licenserc.yaml deleted file mode 100644 index fd8411e049..0000000000 --- a/pulsaradmin/.licenserc.yaml +++ /dev/null @@ -1,29 +0,0 @@ -# Copyright 2023 StreamNative, Inc. -# -# 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. - -header: - license: - spdx-id: Apache-2.0 - copyright-owner: StreamNative, Inc. - copyright-year: '2023' - software-name: pulsar-admin-go - - paths-ignore: - - '**/go.mod' - - '**/go.sum' - - 'LICENSE' - - 'NOTICE' - - PULL_REQUEST_TEMPLATE.md - - comment: on-failure diff --git a/pulsaradmin/CONTRIBUTING.md b/pulsaradmin/CONTRIBUTING.md index f5d9532574..6629479299 100644 --- a/pulsaradmin/CONTRIBUTING.md +++ b/pulsaradmin/CONTRIBUTING.md @@ -1,43 +1,23 @@ - - # Contributing guidelines ## Project structure + The overall project structure is illustrated below: -```shell +```text ├── pkg/ -│   ├── admin/ -│   │   ├── auth/ -│   │   ├── config/ -│   ├── rest/ -│   └── utils/ -├── alias.go -├── go.mod -└── go.sum +│ ├── admin/ +│ │ ├── auth/ +│ │ ├── config/ +│ ├── rest/ +│ └── utils/ +└── alias.go ``` -- The `alias.go` file in the root defines `pulsaradmin` package scope, which contains shortcuts of some types and functions from the `pkg`. +- The `alias.go` file in the root defines `pulsaradmin` package scope, which contains shortcuts of some types and functions from the `pkg`. - The `pkg/admin` package contains all operations for pulsar admin resources. *Note: We should add a new file here if we wanna support a new resource.* - The `pkg/admin/config` package contains configuration options for constructing a pulsar admin client. - - The `pkg/admin/auth` package contains auth providers which work in transport layer. + - The `pkg/admin/auth` package contains auth providers which work in transport layer. - The `pkg/rest` package contains a wrapped HTTP client for requesting pulsar REST API. - The `pkg/utils` package contains common data structures and functions. @@ -67,14 +47,14 @@ Please read through below conventions before contributions. - Please consider package name when selecting an interface name, and avoid redundancy. For example, `storage.Interface` is better than `storage.StorageInterface`. - Do not use uppercase characters, underscores, or dashes in package names. - Please consider parent directory name when choosing a package name. For example, `pkg/controllers/autoscaler/foo.go` should say `package autoscaler` not `package autoscalercontroller`. - - Unless there's a good reason, the `package foo` line should match the name of the directory in which the `.go` file exists. - - Importers can use a different name if they need to disambiguate. + - Unless there's a good reason, the `package foo` line should match the name of the directory in which the `.go` file exists. + - Importers can use a different name if they need to disambiguate. - Locks should be called `lock` and should never be embedded (always `lock sync.Mutex`). When multiple locks are present, give each lock a distinct name following Go conventions: `stateLock`, `mapLock` etc. - + ### Folder and file conventions - All filenames should be lowercase. - Go source files and directories use underscores, not dashes. - Package directories should generally avoid using separators as much as possible. When package names are multiple words, they usually should be in nested subdirectories. - Document directories and filenames should use dashes rather than underscores. -- All source files should add a license at the beginning. \ No newline at end of file +- All source files should add a license at the beginning. diff --git a/pulsaradmin/LICENSE b/pulsaradmin/LICENSE deleted file mode 100644 index d645695673..0000000000 --- a/pulsaradmin/LICENSE +++ /dev/null @@ -1,202 +0,0 @@ - - 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/pulsaradmin/Makefile b/pulsaradmin/Makefile deleted file mode 100644 index c6626aa42c..0000000000 --- a/pulsaradmin/Makefile +++ /dev/null @@ -1,53 +0,0 @@ -# Copyright 2023 StreamNative, Inc. -# -# 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. - -.PHONY: all -all: license-check lint test - -.PHONY: lint -lint: golangci-lint - $(GOLANGCI_LINT) run - -.PHONY: test -test: - @go build ./... && go test -race ./... - -.PHONY: license-check -license-check: license-eye - $(LICENSE_EYE) header check - -.PHONY: license-fix -license-fix: - $(LICENSE_EYE) header fix - -# Install development dependencies - -ifeq (,$(shell go env GOBIN)) -GOBIN=$(shell go env GOPATH)/bin -else -GOBIN=$(shell go env GOBIN) -endif - -LICENSE_EYE ?= $(GOBIN)/license-eye -GOLANGCI_LINT ?= $(GOBIN)/golangci-lint - -.PHONY: license-eye -license-eye: $(LICENSE_EYE) -$(LICENSE_EYE): $(GOBIN) - test -s $(GOBIN)/license-eye || go install github.com/apache/skywalking-eyes/cmd/license-eye@e1a0235 - -.PHONY: golangci-lint -golangci-lint: $(GOLANGCI_LINT) -$(GOLANGCI_LINT): $(GOBIN) - test -s $(GOBIN)/golangci-lint || go install github.com/golangci/golangci-lint/cmd/golangci-lint@v1.51.2 diff --git a/pulsaradmin/README.md b/pulsaradmin/README.md index 7787601ea2..bf5ca739ca 100644 --- a/pulsaradmin/README.md +++ b/pulsaradmin/README.md @@ -1,37 +1,13 @@ - - # Pulsar Admin Go Library -[![Go Reference](https://pkg.go.dev/badge/github.com/streamnative/pulsar-admin-go.svg)](https://pkg.go.dev/github.com/streamnative/pulsar-admin-go) -[![Go Report Card](https://goreportcard.com/badge/github.com/streamnative/pulsar-admin-go)](https://goreportcard.com/report/github.com/streamnative/pulsar-admin-go) -[![Language](https://img.shields.io/badge/Language-Go-blue.svg)](https://golang.org/) -[![License](https://img.shields.io/badge/License-Apache_2.0-blue.svg)](/~https://github.com/streamnative/pulsar-admin-go/blob/master/LICENSE) - Pulsar-Admin-Go is a [Go](https://go.dev) library for [Apache Pulsar](https://pulsar.apache.org/). It provides a unified Go API for managing pulsar resources such as tenants, namespaces and topics, etc. ## Motivation -Currently many projects (e.g, [terraform-provider-pulsar](/~https://github.com/streamnative/terraform-provider-pulsar) and [pulsar-resources-operator](/~https://github.com/streamnative/pulsar-resources-operator)) -that need to manipulate the pulsar admin resources rely on the [pulsarctl](/~https://github.com/streamnative/pulsarctl), +Currently, many projects (e.g, [terraform-provider-pulsar](/~https://github.com/streamnative/terraform-provider-pulsar) and [pulsar-resources-operator](/~https://github.com/streamnative/pulsar-resources-operator)) +that need to manipulate the pulsar admin resources rely on the [pulsarctl](/~https://github.com/streamnative/pulsarctl), which poses challenges for dependency management and versioning as we have to release a new pulsarctl to get updates. -So we decoupled the pulsar admin related api from pulsarctl and created the [pulsar-admin-go](/~https://github.com/streamnative/pulsar-admin-go) library based on it, +So we decoupled the pulsar admin related api from pulsarctl and created the [pulsar-admin-go](/~https://github.com/apache/pulsar-client-go/pulsaradmin) library based on it, which also provides a clearer perspective and maintainability from an architectural perspective. ## Quickstart @@ -42,7 +18,7 @@ which also provides a clearer perspective and maintainability from an architectu - pulsar-admin-go in go.mod ```shell - go get github.com/streamnative/pulsar-admin-go + go get github.com/apache/pulsar-client-go ``` ### Manage pulsar tenants @@ -51,7 +27,7 @@ which also provides a clearer perspective and maintainability from an architectu ```go import ( - "github.com/streamnative/pulsar-admin-go" + "github.com/apache/pulsar-client-go/pulsaradmin" ) func main() { @@ -60,7 +36,7 @@ func main() { if err != nil { panic(err) } - + tenants, _ := admin.Tenants().List() } ``` @@ -71,7 +47,7 @@ func main() { ```go import ( - "github.com/streamnative/pulsar-admin-go" + "github.com/apache/pulsar-client-go/pulsaradmin" ) func main() { @@ -80,7 +56,7 @@ func main() { if err != nil { panic(err) } - + namespaces, _ := admin.Namespaces().GetNamespaces("public") } ``` @@ -89,7 +65,7 @@ func main() { ```go import ( - "github.com/streamnative/pulsar-admin-go" + "github.com/apache/pulsar-client-go/pulsaradmin" ) func main() { @@ -98,7 +74,7 @@ func main() { if err != nil { panic(err) } - + admin.Namespaces().CreateNamespace("public/dev") } ``` @@ -110,8 +86,8 @@ func main() { ```go import ( - "github.com/streamnative/pulsar-admin-go" - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) func main() { @@ -120,19 +96,19 @@ func main() { if err != nil { panic(err) } - + topic, _ := utils.GetTopicName("public/dev/topic") - + admin.Topics().Create(*topic, 3) } ``` ## Contributing -Contributions are warmly welcomed and greatly appreciated! -The project follows the typical GitHub pull request model. See [CONTRIBUTING.md](CONTRIBUTING.md) for more details. +Contributions are warmly welcomed and greatly appreciated! +The project follows the typical GitHub pull request model. See [CONTRIBUTING.md](CONTRIBUTING.md) for more details. Before starting any work, please either comment on an existing issue, or file a new one. ## License -Licensed under the [Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0). See [LICENSE](LICENSE) \ No newline at end of file +Licensed under the [Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0). diff --git a/pulsaradmin/alias.go b/pulsaradmin/alias.go index fc734d69e7..b699c35280 100644 --- a/pulsaradmin/alias.go +++ b/pulsaradmin/alias.go @@ -16,8 +16,8 @@ package pulsaradmin import ( - "github.com/streamnative/pulsar-admin-go/pkg/admin" - "github.com/streamnative/pulsar-admin-go/pkg/admin/config" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" ) // Client contains all admin interfaces for operating pulsar resources diff --git a/pulsaradmin/go.mod b/pulsaradmin/go.mod deleted file mode 100644 index a8ec585fc7..0000000000 --- a/pulsaradmin/go.mod +++ /dev/null @@ -1,32 +0,0 @@ -module github.com/streamnative/pulsar-admin-go - -go 1.18 - -require ( - github.com/99designs/keyring v1.2.1 - github.com/apache/pulsar-client-go v0.9.0 - github.com/golang/protobuf v1.5.2 - github.com/pkg/errors v0.9.1 - github.com/stretchr/testify v1.8.0 - golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 -) - -require ( - github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect - github.com/danieljoos/wincred v1.1.2 // indirect - github.com/davecgh/go-spew v1.1.1 // indirect - github.com/dvsekhvalnov/jose2go v1.5.0 // indirect - github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect - github.com/golang-jwt/jwt v3.2.1+incompatible // indirect - github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect - github.com/mtibben/percent v0.2.1 // indirect - github.com/pmezard/go-difflib v1.0.0 // indirect - golang.org/x/net v0.7.0 // indirect - golang.org/x/sys v0.5.0 // indirect - golang.org/x/term v0.5.0 // indirect - google.golang.org/appengine v1.6.7 // indirect - google.golang.org/protobuf v1.26.0 // indirect - gopkg.in/yaml.v3 v3.0.1 // indirect -) - -replace golang.org/x/sys => golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 diff --git a/pulsaradmin/go.sum b/pulsaradmin/go.sum deleted file mode 100644 index 124611f102..0000000000 --- a/pulsaradmin/go.sum +++ /dev/null @@ -1,682 +0,0 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= -cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= -cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= -cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= -cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= -cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= -cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= -cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= -cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bPc= -cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= -cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= -cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= -cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= -cloud.google.com/go v0.72.0/go.mod h1:M+5Vjvlc2wnp6tjzE102Dw08nGShTscUx2nZMufOKPI= -cloud.google.com/go v0.74.0/go.mod h1:VV1xSbzvo+9QJOxLDaJfTjx5e+MePCpCWwvftOeQmWk= -cloud.google.com/go v0.78.0/go.mod h1:QjdrLG0uq+YwhjoVOLsS1t7TW8fs36kLs4XO5R5ECHg= -cloud.google.com/go v0.79.0/go.mod h1:3bzgcEeQlzbuEAYu4mrWhKqWjmpprinYgKJLgKHnbb8= -cloud.google.com/go v0.81.0/go.mod h1:mk/AM35KwGk/Nm2YSeZbxXdrNK3KZOYHmLkOqC2V6E0= -cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= -cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= -cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= -cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= -cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= -cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= -cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= -cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= -cloud.google.com/go/firestore v1.1.0/go.mod h1:ulACoGHTpvq5r8rxGJ4ddJZBZqakUQqClKRT5SZwBmk= -cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= -cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= -cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= -cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= -cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= -cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= -cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= -cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= -cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= -dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= -github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 h1:/vQbFIOMbk2FiG/kXiLl8BRyzTWDw7gX/Hz7Dd5eDMs= -github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4/go.mod h1:hN7oaIRCjzsZ2dE+yG5k+rsdt3qcwykqK6HVGcKwsw4= -github.com/99designs/keyring v1.2.1 h1:tYLp1ULvO7i3fI5vE21ReQuj99QFSs7lGm0xWyJo87o= -github.com/99designs/keyring v1.2.1/go.mod h1:fc+wB5KTk9wQ9sDx0kFXB3A0MaeGHM9AwRStKOQ5vOA= -github.com/AthenZ/athenz v1.10.39/go.mod h1:3Tg8HLsiQZp81BJY58JBeU2BR6B/H4/0MQGfCwhHNEA= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/DataDog/zstd v1.5.0/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= -github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= -github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= -github.com/apache/pulsar-client-go v0.9.0 h1:L5jvGFXJm0JNA/PgUiJctTVHHttCe4wIEFDv4vojiQM= -github.com/apache/pulsar-client-go v0.9.0/go.mod h1:fSAcBipgz4KQ/VgwZEJtQ71cCXMKm8ezznstrozrngw= -github.com/ardielle/ardielle-go v1.5.2/go.mod h1:I4hy1n795cUhaVt/ojz83SNVCYIGsAFAONtv2Dr7HUI= -github.com/ardielle/ardielle-tools v1.5.4/go.mod h1:oZN+JRMnqGiIhrzkRN9l26Cej9dEx4jeNG6A+AdkShk= -github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= -github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= -github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= -github.com/aws/aws-sdk-go v1.32.6/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= -github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= -github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= -github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= -github.com/bketelsen/crypt v0.0.4/go.mod h1:aI6NrJ0pMGgvZKL1iVgXLnfIFJtfV+bKCoqOes/6LfM= -github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b/go.mod h1:ac9efd0D1fsDb3EJvhqgXRbFx7bs2wqZ10HQPeU8U/Q= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= -github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= -github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= -github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= -github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= -github.com/danieljoos/wincred v1.1.2 h1:QLdCxFs1/Yl4zduvBdcHB8goaYk9RARS2SgLLRuAyr0= -github.com/danieljoos/wincred v1.1.2/go.mod h1:GijpziifJoIBfYh+S7BbkdUTU4LfM+QnGqR5Vl2tAx0= -github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= -github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dimfeld/httptreemux v5.0.1+incompatible/go.mod h1:rbUlSV+CCpv/SuqUTP/8Bk2O3LyUV436/yaRGkhP6Z0= -github.com/dvsekhvalnov/jose2go v1.5.0 h1:3j8ya4Z4kMCwT5nXIKFSV84YS+HdqSSO0VsTQxaLAeM= -github.com/dvsekhvalnov/jose2go v1.5.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/go-control-plane v0.9.7/go.mod h1:cwu0lG7PUMfa9snN8LXBig5ynNVH9qI8YYLbd1fK2po= -github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= -github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= -github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= -github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= -github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= -github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= -github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= -github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= -github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= -github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= -github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 h1:ZpnhV/YsD2/4cESfV5+Hoeu/iUR3ruzNvZ+yQfO03a0= -github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2/go.mod h1:bBOAhwG1umN6/6ZUMtDFBMQR8jRg9O75tm9K00oMsK4= -github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= -github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang-jwt/jwt v3.2.1+incompatible h1:73Z+4BJcrTC+KczS6WvTPvRGOp1WmfEP4Q1lOd9Z/+c= -github.com/golang-jwt/jwt v3.2.1+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= -github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= -github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= -github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= -github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= -github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= -github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= -github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= -github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= -github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= -github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= -github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.1/go.mod h1:DopwsBzvsk0Fs44TXzsVbJyPhcCPeIwnvohx4u74HPM= -github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= -github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= -github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= -github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= -github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= -github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= -github.com/google/martian/v3 v3.1.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= -github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20201023163331-3e6fc7fc9c4c/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210407192527-94a9f03dee38/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= -github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= -github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= -github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= -github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= -github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c h1:6rhixN/i8ZofjG1Y75iExal34USq5p+wiN1tpie8IrU= -github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c/go.mod h1:NMPJylDgVpX0MLRlPy15sqSwOFv/U1GZ2m21JhFfek0= -github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q= -github.com/hashicorp/consul/sdk v0.1.1/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= -github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= -github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= -github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= -github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= -github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= -github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= -github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= -github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= -github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= -github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= -github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= -github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= -github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= -github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= -github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= -github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= -github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= -github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/jawher/mow.cli v1.0.4/go.mod h1:5hQj2V8g+qYmLUVWqu4Wuja1pI57M83EChYLVZ0sMKk= -github.com/jawher/mow.cli v1.2.0/go.mod h1:y+pcA3jBAdo/GIZx/0rFjw/K2bVEODP9rfZOfaiq8Ko= -github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= -github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= -github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= -github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= -github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= -github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= -github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= -github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.14.4/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= -github.com/linkedin/goavro/v2 v2.9.8/go.mod h1:UgQUb2N/pmueQYH9bfqFioWxzYCZXSfF8Jw03O5sjqA= -github.com/magiconair/properties v1.8.5/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= -github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= -github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= -github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= -github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= -github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= -github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= -github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= -github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= -github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= -github.com/mitchellh/mapstructure v1.4.1/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= -github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/mtibben/percent v0.2.1 h1:5gssi8Nqo8QU/r2pynCm+hBQHpkB/uNK7BJCFogWdzs= -github.com/mtibben/percent v0.2.1/go.mod h1:KG9uO+SZkUp+VkRHsCdYQV3XSZrrSpR3O9ibNBTZrns= -github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= -github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= -github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= -github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= -github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= -github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= -github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= -github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= -github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= -github.com/onsi/ginkgo/v2 v2.1.3/go.mod h1:vw5CSIxN1JObi/U8gcbwft7ZxR2dgaR70JSE3/PpL4c= -github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= -github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= -github.com/onsi/gomega v1.17.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY= -github.com/onsi/gomega v1.19.0 h1:4ieX6qQjPP/BfC3mpsAtIGGlxTWPeA3Inl/7DtXw1tw= -github.com/onsi/gomega v1.19.0/go.mod h1:LY+I3pBVzYsTBU1AnDwOSxaYi9WoWiqgwooUqq9yPro= -github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= -github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= -github.com/pelletier/go-toml v1.9.3/go.mod h1:u1nR/EPcESfeI/szUZKdtJ0xRNbUoANCkoOuaOx1Y+c= -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= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= -github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= -github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= -github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= -github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= -github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.11.1/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= -github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= -github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= -github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= -github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= -github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= -github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= -github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= -github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= -github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= -github.com/spf13/cast v1.3.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v1.2.1/go.mod h1:ExllRjgxM/piMAM+3tAZvg8fsklGAf3tPfi+i8t68Nk= -github.com/spf13/jwalterweatherman v1.1.0/go.mod h1:aNWZUN0dPAAO/Ljvb5BEdw96iTZ0EXowPYD95IqWIGo= -github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= -github.com/spf13/viper v1.8.1/go.mod h1:o0Pch8wJ9BVSWGQMbra6iw0oQ5oktSIBaujf1rJH9Ns= -github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= -github.com/stretchr/objx v0.3.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= -github.com/stretchr/objx v0.4.0 h1:M2gUjqZET1qApGOWNSnZ49BAIMX4F/1plDv3+l31EJ4= -github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= -github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= -github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PKk= -github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= -github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.etcd.io/etcd/api/v3 v3.5.0/go.mod h1:cbVKeC6lCfl7j/8jBhAK6aIYO9XOjdptoxU/nLQcPvs= -go.etcd.io/etcd/client/pkg/v3 v3.5.0/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= -go.etcd.io/etcd/client/v2 v2.305.0/go.mod h1:h9puh54ZTgAKtEbut2oe9P4L/oqKCVB6xsXlzd7alYQ= -go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= -go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= -go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= -go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/zap v1.17.0/go.mod h1:MXVU+bhUf/A7Xi2HNOnopQOrmycQ5Ih87HtOu4q5SSo= -golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= -golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= -golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= -golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= -golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= -golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= -golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20201208152925-83fdc39ff7b5/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= -golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= -golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= -golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= -golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= -golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= -golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= -golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.7.0 h1:rJrUqqhjsgNp7KqAIc25s9pZnjU7TUcSY7HcVZjdn1g= -golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210313182246-cd4f82c27b84/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 h1:0Ja1LBD+yisY6RWM/BH7TJVXWsSjs2VwBSmvSX4HdBc= -golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 h1:xHms4gcpe1YE7A3yIllJXP16CMAGuqwO2lX1mTyyRRc= -golang.org/x/sys v0.0.0-20220422013727-9388b58f7150/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.5.0 h1:n2a8QNdAb0sZNpU9R1ALUXBbY+w51fCQDN+7EdxNBsY= -golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= -golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= -golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.7.0 h1:4BRB4x83lYWy72KwLD/qYDuTu7q9PjSagHvijDw7cLo= -golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191112195655-aa38f8e97acc/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200227222343-706bc42d1f0d/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= -golang.org/x/tools v0.0.0-20200312045724-11d5b4c81c7d/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= -golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= -golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200904185747-39188db58858/go.mod h1:Cj7w3i3Rnn0Xh82ur9kSqwfTHTeVxaDqrfMjpcNT6bE= -golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= -golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= -golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= -google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= -google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= -google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= -google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.19.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= -google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= -google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= -google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= -google.golang.org/api v0.35.0/go.mod h1:/XrVsuzM0rZmrsbjJutiuftIzeuTQcEeaYcSk/mQ1dg= -google.golang.org/api v0.36.0/go.mod h1:+z5ficQTmoYpPn8LCUNVpK5I7hwkpjbcgqA7I34qYtE= -google.golang.org/api v0.40.0/go.mod h1:fYKFpnQN0DsDSKRVRcQSDQNtqWPfM9i+zNPxepjRCQ8= -google.golang.org/api v0.41.0/go.mod h1:RkxM5lITDfTzmyKFPt+wGrCJbVfniCr2ool8kTBzRTU= -google.golang.org/api v0.43.0/go.mod h1:nQsDGjRXMo4lvh5hP0TKqF244gqhGcr/YSIykhUk/94= -google.golang.org/api v0.44.0/go.mod h1:EBOGZqzyhtvMDoxwS97ctnh0zUmYY6CxqXsc1AvkYD8= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= -google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6c= -google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= -google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= -google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= -google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= -google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200904004341-0bd0a958aa1d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201109203340-2640f1f9cdfb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201201144952-b05cb90ed32e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201210142538-e3217bee35cc/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201214200347-8c77b98c765d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210222152913-aa3ee6e6a81c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210303154014-9728d6b83eeb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210310155132-4ce2db91004e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210319143718-93e7006c17a6/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210402141018-6c239bbf2bb1/go.mod h1:9lPAdzaEmUacj36I+k7YKbEc5CXzPIeORRgDAUOu28A= -google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= -google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.31.1/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= -google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA51WJ8= -google.golang.org/grpc v1.35.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.36.1/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= -google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= -google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= -google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= -google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= -google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= -google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= -google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= -google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0 h1:bxAC2xTBsZGibn2RTntX0oH50xLsqy1OxA9tTL3p/lk= -google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= -gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLFVxaq6wH4YuVdsUOr75U= -gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= -gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= -gopkg.in/ini.v1 v1.62.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= -gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= -gopkg.in/square/go-jose.v2 v2.4.1/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= -gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= -gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= -gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= -gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= -gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= -rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= -rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/pulsaradmin/pkg/admin/admin.go b/pulsaradmin/pkg/admin/admin.go index 74d6071672..e9454407e6 100644 --- a/pulsaradmin/pkg/admin/admin.go +++ b/pulsaradmin/pkg/admin/admin.go @@ -21,10 +21,10 @@ import ( "path" "time" - "github.com/streamnative/pulsar-admin-go/pkg/admin/auth" - "github.com/streamnative/pulsar-admin-go/pkg/admin/config" - "github.com/streamnative/pulsar-admin-go/pkg/rest" - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/auth" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/rest" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) const ( diff --git a/pulsaradmin/pkg/admin/admin_test.go b/pulsaradmin/pkg/admin/admin_test.go index d7bd457210..c8be77bbfe 100644 --- a/pulsaradmin/pkg/admin/admin_test.go +++ b/pulsaradmin/pkg/admin/admin_test.go @@ -22,8 +22,8 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/streamnative/pulsar-admin-go/pkg/admin/auth" - "github.com/streamnative/pulsar-admin-go/pkg/admin/config" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/auth" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" ) func TestPulsarClientEndpointEscapes(t *testing.T) { diff --git a/pulsaradmin/pkg/admin/auth/provider.go b/pulsaradmin/pkg/admin/auth/provider.go index 1b25ec8534..b88677c473 100644 --- a/pulsaradmin/pkg/admin/auth/provider.go +++ b/pulsaradmin/pkg/admin/auth/provider.go @@ -20,7 +20,7 @@ import ( "github.com/apache/pulsar-client-go/oauth2" - "github.com/streamnative/pulsar-admin-go/pkg/admin/config" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" ) // Provider provide a general method to add auth message diff --git a/pulsaradmin/pkg/admin/auth/transport.go b/pulsaradmin/pkg/admin/auth/transport.go index 0ed2efd61e..170d91ac7c 100644 --- a/pulsaradmin/pkg/admin/auth/transport.go +++ b/pulsaradmin/pkg/admin/auth/transport.go @@ -21,7 +21,7 @@ import ( "net/http" "os" - "github.com/streamnative/pulsar-admin-go/pkg/admin/config" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" ) type Transport struct { diff --git a/pulsaradmin/pkg/admin/broker_stats.go b/pulsaradmin/pkg/admin/broker_stats.go index f2912470e2..8e35d16dd1 100644 --- a/pulsaradmin/pkg/admin/broker_stats.go +++ b/pulsaradmin/pkg/admin/broker_stats.go @@ -16,7 +16,7 @@ package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // BrokerStats is admin interface for broker stats management diff --git a/pulsaradmin/pkg/admin/brokers.go b/pulsaradmin/pkg/admin/brokers.go index 6b6a4db4b1..4ce5c4bc4e 100644 --- a/pulsaradmin/pkg/admin/brokers.go +++ b/pulsaradmin/pkg/admin/brokers.go @@ -20,7 +20,7 @@ import ( "net/url" "strings" - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // Brokers is admin interface for brokers management diff --git a/pulsaradmin/pkg/admin/cluster.go b/pulsaradmin/pkg/admin/cluster.go index 59670899e0..5c12c9b844 100644 --- a/pulsaradmin/pkg/admin/cluster.go +++ b/pulsaradmin/pkg/admin/cluster.go @@ -16,7 +16,7 @@ package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // Clusters is admin interface for clusters management diff --git a/pulsaradmin/pkg/admin/functions.go b/pulsaradmin/pkg/admin/functions.go index 0150b71320..272af95f3b 100644 --- a/pulsaradmin/pkg/admin/functions.go +++ b/pulsaradmin/pkg/admin/functions.go @@ -26,7 +26,7 @@ import ( "path/filepath" "strings" - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // Functions is admin interface for functions management diff --git a/pulsaradmin/pkg/admin/functions_worker.go b/pulsaradmin/pkg/admin/functions_worker.go index 756c76eb7b..3a0345a0f4 100644 --- a/pulsaradmin/pkg/admin/functions_worker.go +++ b/pulsaradmin/pkg/admin/functions_worker.go @@ -16,7 +16,7 @@ package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) type FunctionsWorker interface { diff --git a/pulsaradmin/pkg/admin/namespace.go b/pulsaradmin/pkg/admin/namespace.go index 5b1f0dfe93..b09feb4209 100644 --- a/pulsaradmin/pkg/admin/namespace.go +++ b/pulsaradmin/pkg/admin/namespace.go @@ -20,7 +20,7 @@ import ( "strconv" "strings" - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // Namespaces is admin interface for namespaces management diff --git a/pulsaradmin/pkg/admin/ns_isolation_policy.go b/pulsaradmin/pkg/admin/ns_isolation_policy.go index 993f110ce5..ea4ab2ee1f 100644 --- a/pulsaradmin/pkg/admin/ns_isolation_policy.go +++ b/pulsaradmin/pkg/admin/ns_isolation_policy.go @@ -16,7 +16,7 @@ package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) type NsIsolationPolicy interface { diff --git a/pulsaradmin/pkg/admin/packages.go b/pulsaradmin/pkg/admin/packages.go index 67544b5eeb..24ff97de3a 100644 --- a/pulsaradmin/pkg/admin/packages.go +++ b/pulsaradmin/pkg/admin/packages.go @@ -29,7 +29,7 @@ import ( "github.com/pkg/errors" - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // Packages is admin interface for functions management diff --git a/pulsaradmin/pkg/admin/resource_quotas.go b/pulsaradmin/pkg/admin/resource_quotas.go index 24b5d5cf90..5fec98326f 100644 --- a/pulsaradmin/pkg/admin/resource_quotas.go +++ b/pulsaradmin/pkg/admin/resource_quotas.go @@ -16,7 +16,7 @@ package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) type ResourceQuotas interface { diff --git a/pulsaradmin/pkg/admin/schema.go b/pulsaradmin/pkg/admin/schema.go index 27794f9756..461b91997a 100644 --- a/pulsaradmin/pkg/admin/schema.go +++ b/pulsaradmin/pkg/admin/schema.go @@ -19,7 +19,7 @@ import ( "fmt" "strconv" - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // Schema is admin interface for schema management diff --git a/pulsaradmin/pkg/admin/sinks.go b/pulsaradmin/pkg/admin/sinks.go index 6c85f414ca..1af650d79f 100644 --- a/pulsaradmin/pkg/admin/sinks.go +++ b/pulsaradmin/pkg/admin/sinks.go @@ -26,7 +26,7 @@ import ( "path/filepath" "strings" - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // Sinks is admin interface for sinks management diff --git a/pulsaradmin/pkg/admin/sources.go b/pulsaradmin/pkg/admin/sources.go index 952155f034..c02cbb94dc 100644 --- a/pulsaradmin/pkg/admin/sources.go +++ b/pulsaradmin/pkg/admin/sources.go @@ -26,7 +26,7 @@ import ( "path/filepath" "strings" - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // Sources is admin interface for sources management diff --git a/pulsaradmin/pkg/admin/subscription.go b/pulsaradmin/pkg/admin/subscription.go index ba2f45a49e..02f48e6c18 100644 --- a/pulsaradmin/pkg/admin/subscription.go +++ b/pulsaradmin/pkg/admin/subscription.go @@ -26,7 +26,7 @@ import ( "github.com/golang/protobuf/proto" //nolint:staticcheck - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // Subscriptions is admin interface for subscriptions management diff --git a/pulsaradmin/pkg/admin/tenant.go b/pulsaradmin/pkg/admin/tenant.go index b536da2d4a..af287a646d 100644 --- a/pulsaradmin/pkg/admin/tenant.go +++ b/pulsaradmin/pkg/admin/tenant.go @@ -16,7 +16,7 @@ package admin import ( - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // Tenants is admin interface for tenants management diff --git a/pulsaradmin/pkg/admin/topic.go b/pulsaradmin/pkg/admin/topic.go index 546017ada1..0bbff0000b 100644 --- a/pulsaradmin/pkg/admin/topic.go +++ b/pulsaradmin/pkg/admin/topic.go @@ -19,7 +19,7 @@ import ( "fmt" "strconv" - "github.com/streamnative/pulsar-admin-go/pkg/utils" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) // Topics is admin interface for topics management From 71afeb2e2f92132d3f1faf023378f37332216739 Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 16 Aug 2023 00:08:53 +0800 Subject: [PATCH 270/348] replace license header Signed-off-by: tison --- pulsaradmin/alias.go | 12 +++++++----- pulsaradmin/pkg/admin/admin.go | 12 +++++++----- pulsaradmin/pkg/admin/admin_test.go | 12 +++++++----- pulsaradmin/pkg/admin/auth/oauth2.go | 12 +++++++----- pulsaradmin/pkg/admin/auth/oauth2_test.go | 12 +++++++----- pulsaradmin/pkg/admin/auth/provider.go | 12 +++++++----- pulsaradmin/pkg/admin/auth/tls.go | 12 +++++++----- pulsaradmin/pkg/admin/auth/token.go | 12 +++++++----- pulsaradmin/pkg/admin/auth/transport.go | 12 +++++++----- pulsaradmin/pkg/admin/broker_stats.go | 12 +++++++----- pulsaradmin/pkg/admin/brokers.go | 12 +++++++----- pulsaradmin/pkg/admin/cluster.go | 12 +++++++----- pulsaradmin/pkg/admin/config/api_version.go | 12 +++++++----- pulsaradmin/pkg/admin/config/api_version_test.go | 12 +++++++----- pulsaradmin/pkg/admin/config/config.go | 12 +++++++----- pulsaradmin/pkg/admin/functions.go | 12 +++++++----- pulsaradmin/pkg/admin/functions_worker.go | 12 +++++++----- pulsaradmin/pkg/admin/namespace.go | 12 +++++++----- pulsaradmin/pkg/admin/ns_isolation_policy.go | 12 +++++++----- pulsaradmin/pkg/admin/packages.go | 12 +++++++----- pulsaradmin/pkg/admin/resource_quotas.go | 12 +++++++----- pulsaradmin/pkg/admin/schema.go | 12 +++++++----- pulsaradmin/pkg/admin/sinks.go | 12 +++++++----- pulsaradmin/pkg/admin/sources.go | 12 +++++++----- pulsaradmin/pkg/admin/subscription.go | 12 +++++++----- pulsaradmin/pkg/admin/tenant.go | 12 +++++++----- pulsaradmin/pkg/admin/topic.go | 12 +++++++----- pulsaradmin/pkg/rest/client.go | 12 +++++++----- pulsaradmin/pkg/rest/client_test.go | 12 +++++++----- pulsaradmin/pkg/rest/errors.go | 12 +++++++----- pulsaradmin/pkg/utils/allocator_stats.go | 12 +++++++----- pulsaradmin/pkg/utils/auth_action.go | 12 +++++++----- pulsaradmin/pkg/utils/auth_polices.go | 12 +++++++----- pulsaradmin/pkg/utils/auth_polices_test.go | 12 +++++++----- pulsaradmin/pkg/utils/backlog_quota.go | 12 +++++++----- pulsaradmin/pkg/utils/batch_source_config.go | 12 +++++++----- pulsaradmin/pkg/utils/broker_ns_isolation_data.go | 12 +++++++----- pulsaradmin/pkg/utils/bundles_data.go | 12 +++++++----- pulsaradmin/pkg/utils/connector_definition.go | 12 +++++++----- pulsaradmin/pkg/utils/consumer_config.go | 12 +++++++----- pulsaradmin/pkg/utils/crypto_config.go | 12 +++++++----- pulsaradmin/pkg/utils/data.go | 12 +++++++----- pulsaradmin/pkg/utils/dispatch_rate.go | 12 +++++++----- pulsaradmin/pkg/utils/function_confg.go | 12 +++++++----- pulsaradmin/pkg/utils/function_state.go | 12 +++++++----- pulsaradmin/pkg/utils/function_status.go | 12 +++++++----- pulsaradmin/pkg/utils/functions_stats.go | 12 +++++++----- pulsaradmin/pkg/utils/home_dir.go | 12 +++++++----- pulsaradmin/pkg/utils/inactive_topic_policies.go | 12 +++++++----- pulsaradmin/pkg/utils/internal_configuration_data.go | 12 +++++++----- pulsaradmin/pkg/utils/load_manage_report.go | 12 +++++++----- pulsaradmin/pkg/utils/long_running_process_status.go | 12 +++++++----- pulsaradmin/pkg/utils/message.go | 12 +++++++----- pulsaradmin/pkg/utils/message_id.go | 12 +++++++----- pulsaradmin/pkg/utils/message_id_test.go | 12 +++++++----- pulsaradmin/pkg/utils/metrics.go | 12 +++++++----- pulsaradmin/pkg/utils/namespace_name.go | 12 +++++++----- pulsaradmin/pkg/utils/namespace_name_test.go | 12 +++++++----- pulsaradmin/pkg/utils/ns_isolation_data.go | 12 +++++++----- pulsaradmin/pkg/utils/ns_ownership_status.go | 12 +++++++----- pulsaradmin/pkg/utils/package_metadata.go | 12 +++++++----- pulsaradmin/pkg/utils/package_name.go | 12 +++++++----- pulsaradmin/pkg/utils/package_name_test.go | 12 +++++++----- pulsaradmin/pkg/utils/package_type.go | 12 +++++++----- pulsaradmin/pkg/utils/persistence_policies.go | 12 +++++++----- pulsaradmin/pkg/utils/policies.go | 12 +++++++----- pulsaradmin/pkg/utils/producer_config.go | 12 +++++++----- pulsaradmin/pkg/utils/publish_rate.go | 12 +++++++----- pulsaradmin/pkg/utils/resource_quota.go | 12 +++++++----- pulsaradmin/pkg/utils/resources.go | 12 +++++++----- pulsaradmin/pkg/utils/retention_policies.go | 12 +++++++----- pulsaradmin/pkg/utils/schema_strategy.go | 12 +++++++----- pulsaradmin/pkg/utils/schema_util.go | 12 +++++++----- pulsaradmin/pkg/utils/sink_config.go | 12 +++++++----- pulsaradmin/pkg/utils/sink_status.go | 12 +++++++----- pulsaradmin/pkg/utils/source_config.go | 12 +++++++----- pulsaradmin/pkg/utils/source_status.go | 12 +++++++----- pulsaradmin/pkg/utils/subscription_auth_mode.go | 12 +++++++----- pulsaradmin/pkg/utils/topic_auto_creation_config.go | 12 +++++++----- pulsaradmin/pkg/utils/topic_domain.go | 12 +++++++----- pulsaradmin/pkg/utils/topic_name.go | 12 +++++++----- pulsaradmin/pkg/utils/topic_name_test.go | 12 +++++++----- pulsaradmin/pkg/utils/topic_type.go | 12 +++++++----- pulsaradmin/pkg/utils/topics_stats_stream.go | 12 +++++++----- pulsaradmin/pkg/utils/update_options.go | 12 +++++++----- pulsaradmin/pkg/utils/utils.go | 12 +++++++----- pulsaradmin/pkg/utils/utils_test.go | 12 +++++++----- pulsaradmin/pkg/utils/window_confing.go | 12 +++++++----- pulsaradmin/pkg/utils/worker_info.go | 12 +++++++----- 89 files changed, 623 insertions(+), 445 deletions(-) diff --git a/pulsaradmin/alias.go b/pulsaradmin/alias.go index b699c35280..53c8d93e05 100644 --- a/pulsaradmin/alias.go +++ b/pulsaradmin/alias.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/admin.go b/pulsaradmin/pkg/admin/admin.go index e9454407e6..a22b22c62d 100644 --- a/pulsaradmin/pkg/admin/admin.go +++ b/pulsaradmin/pkg/admin/admin.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/admin_test.go b/pulsaradmin/pkg/admin/admin_test.go index c8be77bbfe..c4fa529567 100644 --- a/pulsaradmin/pkg/admin/admin_test.go +++ b/pulsaradmin/pkg/admin/admin_test.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/auth/oauth2.go b/pulsaradmin/pkg/admin/auth/oauth2.go index 9e8fafa202..eab64cace5 100644 --- a/pulsaradmin/pkg/admin/auth/oauth2.go +++ b/pulsaradmin/pkg/admin/auth/oauth2.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/auth/oauth2_test.go b/pulsaradmin/pkg/admin/auth/oauth2_test.go index 87f7842668..b25e576119 100644 --- a/pulsaradmin/pkg/admin/auth/oauth2_test.go +++ b/pulsaradmin/pkg/admin/auth/oauth2_test.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/auth/provider.go b/pulsaradmin/pkg/admin/auth/provider.go index b88677c473..24f74d2650 100644 --- a/pulsaradmin/pkg/admin/auth/provider.go +++ b/pulsaradmin/pkg/admin/auth/provider.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/auth/tls.go b/pulsaradmin/pkg/admin/auth/tls.go index 4a7077df47..7c11fe27e7 100644 --- a/pulsaradmin/pkg/admin/auth/tls.go +++ b/pulsaradmin/pkg/admin/auth/tls.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/auth/token.go b/pulsaradmin/pkg/admin/auth/token.go index 8938cbc23e..ecf57ecc8f 100644 --- a/pulsaradmin/pkg/admin/auth/token.go +++ b/pulsaradmin/pkg/admin/auth/token.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/auth/transport.go b/pulsaradmin/pkg/admin/auth/transport.go index 170d91ac7c..d96ab57f55 100644 --- a/pulsaradmin/pkg/admin/auth/transport.go +++ b/pulsaradmin/pkg/admin/auth/transport.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/broker_stats.go b/pulsaradmin/pkg/admin/broker_stats.go index 8e35d16dd1..c9f9cb01e3 100644 --- a/pulsaradmin/pkg/admin/broker_stats.go +++ b/pulsaradmin/pkg/admin/broker_stats.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/brokers.go b/pulsaradmin/pkg/admin/brokers.go index 4ce5c4bc4e..79fcb092ef 100644 --- a/pulsaradmin/pkg/admin/brokers.go +++ b/pulsaradmin/pkg/admin/brokers.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/cluster.go b/pulsaradmin/pkg/admin/cluster.go index 5c12c9b844..b290e3ef30 100644 --- a/pulsaradmin/pkg/admin/cluster.go +++ b/pulsaradmin/pkg/admin/cluster.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/config/api_version.go b/pulsaradmin/pkg/admin/config/api_version.go index 6c5cb777f7..95f6704580 100644 --- a/pulsaradmin/pkg/admin/config/api_version.go +++ b/pulsaradmin/pkg/admin/config/api_version.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/config/api_version_test.go b/pulsaradmin/pkg/admin/config/api_version_test.go index c35c1ec817..e1dc7bdb69 100644 --- a/pulsaradmin/pkg/admin/config/api_version_test.go +++ b/pulsaradmin/pkg/admin/config/api_version_test.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/config/config.go b/pulsaradmin/pkg/admin/config/config.go index c993fb9b08..9428f3a909 100644 --- a/pulsaradmin/pkg/admin/config/config.go +++ b/pulsaradmin/pkg/admin/config/config.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/functions.go b/pulsaradmin/pkg/admin/functions.go index 272af95f3b..cbaaf6be4e 100644 --- a/pulsaradmin/pkg/admin/functions.go +++ b/pulsaradmin/pkg/admin/functions.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/functions_worker.go b/pulsaradmin/pkg/admin/functions_worker.go index 3a0345a0f4..3cad65de82 100644 --- a/pulsaradmin/pkg/admin/functions_worker.go +++ b/pulsaradmin/pkg/admin/functions_worker.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/namespace.go b/pulsaradmin/pkg/admin/namespace.go index b09feb4209..732441e8c2 100644 --- a/pulsaradmin/pkg/admin/namespace.go +++ b/pulsaradmin/pkg/admin/namespace.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/ns_isolation_policy.go b/pulsaradmin/pkg/admin/ns_isolation_policy.go index ea4ab2ee1f..d8897f9f63 100644 --- a/pulsaradmin/pkg/admin/ns_isolation_policy.go +++ b/pulsaradmin/pkg/admin/ns_isolation_policy.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/packages.go b/pulsaradmin/pkg/admin/packages.go index 24ff97de3a..c7a0fd5ff1 100644 --- a/pulsaradmin/pkg/admin/packages.go +++ b/pulsaradmin/pkg/admin/packages.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/resource_quotas.go b/pulsaradmin/pkg/admin/resource_quotas.go index 5fec98326f..fc5209b54b 100644 --- a/pulsaradmin/pkg/admin/resource_quotas.go +++ b/pulsaradmin/pkg/admin/resource_quotas.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/schema.go b/pulsaradmin/pkg/admin/schema.go index 461b91997a..1465526841 100644 --- a/pulsaradmin/pkg/admin/schema.go +++ b/pulsaradmin/pkg/admin/schema.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/sinks.go b/pulsaradmin/pkg/admin/sinks.go index 1af650d79f..acbf83113b 100644 --- a/pulsaradmin/pkg/admin/sinks.go +++ b/pulsaradmin/pkg/admin/sinks.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/sources.go b/pulsaradmin/pkg/admin/sources.go index c02cbb94dc..e10d1da061 100644 --- a/pulsaradmin/pkg/admin/sources.go +++ b/pulsaradmin/pkg/admin/sources.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/subscription.go b/pulsaradmin/pkg/admin/subscription.go index 02f48e6c18..456de46cdb 100644 --- a/pulsaradmin/pkg/admin/subscription.go +++ b/pulsaradmin/pkg/admin/subscription.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/tenant.go b/pulsaradmin/pkg/admin/tenant.go index af287a646d..62e176bcbc 100644 --- a/pulsaradmin/pkg/admin/tenant.go +++ b/pulsaradmin/pkg/admin/tenant.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/admin/topic.go b/pulsaradmin/pkg/admin/topic.go index 0bbff0000b..c888827bf8 100644 --- a/pulsaradmin/pkg/admin/topic.go +++ b/pulsaradmin/pkg/admin/topic.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/rest/client.go b/pulsaradmin/pkg/rest/client.go index fad7a811af..a1a79737c1 100644 --- a/pulsaradmin/pkg/rest/client.go +++ b/pulsaradmin/pkg/rest/client.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/rest/client_test.go b/pulsaradmin/pkg/rest/client_test.go index 193f5093d1..50a7c5a848 100644 --- a/pulsaradmin/pkg/rest/client_test.go +++ b/pulsaradmin/pkg/rest/client_test.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/rest/errors.go b/pulsaradmin/pkg/rest/errors.go index 7eff7be473..dc611ec388 100644 --- a/pulsaradmin/pkg/rest/errors.go +++ b/pulsaradmin/pkg/rest/errors.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/allocator_stats.go b/pulsaradmin/pkg/utils/allocator_stats.go index 05dc58d54d..4c2a7fb521 100644 --- a/pulsaradmin/pkg/utils/allocator_stats.go +++ b/pulsaradmin/pkg/utils/allocator_stats.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/auth_action.go b/pulsaradmin/pkg/utils/auth_action.go index 91fb3c0688..7f2bf25735 100644 --- a/pulsaradmin/pkg/utils/auth_action.go +++ b/pulsaradmin/pkg/utils/auth_action.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/auth_polices.go b/pulsaradmin/pkg/utils/auth_polices.go index 55ba2983af..065b3c434d 100644 --- a/pulsaradmin/pkg/utils/auth_polices.go +++ b/pulsaradmin/pkg/utils/auth_polices.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/auth_polices_test.go b/pulsaradmin/pkg/utils/auth_polices_test.go index 20326ebbdb..014f594bdc 100644 --- a/pulsaradmin/pkg/utils/auth_polices_test.go +++ b/pulsaradmin/pkg/utils/auth_polices_test.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/backlog_quota.go b/pulsaradmin/pkg/utils/backlog_quota.go index 507a1165ef..1930cad8ce 100644 --- a/pulsaradmin/pkg/utils/backlog_quota.go +++ b/pulsaradmin/pkg/utils/backlog_quota.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/batch_source_config.go b/pulsaradmin/pkg/utils/batch_source_config.go index 1a7ee1830b..3db2db697c 100644 --- a/pulsaradmin/pkg/utils/batch_source_config.go +++ b/pulsaradmin/pkg/utils/batch_source_config.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/broker_ns_isolation_data.go b/pulsaradmin/pkg/utils/broker_ns_isolation_data.go index 342b9b3446..1381647b6b 100644 --- a/pulsaradmin/pkg/utils/broker_ns_isolation_data.go +++ b/pulsaradmin/pkg/utils/broker_ns_isolation_data.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/bundles_data.go b/pulsaradmin/pkg/utils/bundles_data.go index 773df05121..2bba7ae493 100644 --- a/pulsaradmin/pkg/utils/bundles_data.go +++ b/pulsaradmin/pkg/utils/bundles_data.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/connector_definition.go b/pulsaradmin/pkg/utils/connector_definition.go index 0e1569f8cb..72b12893cd 100644 --- a/pulsaradmin/pkg/utils/connector_definition.go +++ b/pulsaradmin/pkg/utils/connector_definition.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/consumer_config.go b/pulsaradmin/pkg/utils/consumer_config.go index 2502edf6df..f609ae48d8 100644 --- a/pulsaradmin/pkg/utils/consumer_config.go +++ b/pulsaradmin/pkg/utils/consumer_config.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/crypto_config.go b/pulsaradmin/pkg/utils/crypto_config.go index e9ed0350a5..e411bb8007 100644 --- a/pulsaradmin/pkg/utils/crypto_config.go +++ b/pulsaradmin/pkg/utils/crypto_config.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/data.go b/pulsaradmin/pkg/utils/data.go index ea9b9046a6..55888aab2b 100644 --- a/pulsaradmin/pkg/utils/data.go +++ b/pulsaradmin/pkg/utils/data.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/dispatch_rate.go b/pulsaradmin/pkg/utils/dispatch_rate.go index 0adfab773a..9c6ccce88c 100644 --- a/pulsaradmin/pkg/utils/dispatch_rate.go +++ b/pulsaradmin/pkg/utils/dispatch_rate.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/function_confg.go b/pulsaradmin/pkg/utils/function_confg.go index ce7017e362..02a00db3b4 100644 --- a/pulsaradmin/pkg/utils/function_confg.go +++ b/pulsaradmin/pkg/utils/function_confg.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/function_state.go b/pulsaradmin/pkg/utils/function_state.go index fae5f581c2..63fa15057c 100644 --- a/pulsaradmin/pkg/utils/function_state.go +++ b/pulsaradmin/pkg/utils/function_state.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/function_status.go b/pulsaradmin/pkg/utils/function_status.go index 66fa45c4b2..360e61b077 100644 --- a/pulsaradmin/pkg/utils/function_status.go +++ b/pulsaradmin/pkg/utils/function_status.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/functions_stats.go b/pulsaradmin/pkg/utils/functions_stats.go index 96984790d3..82761afe51 100644 --- a/pulsaradmin/pkg/utils/functions_stats.go +++ b/pulsaradmin/pkg/utils/functions_stats.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/home_dir.go b/pulsaradmin/pkg/utils/home_dir.go index 3cee68e21c..330831c405 100644 --- a/pulsaradmin/pkg/utils/home_dir.go +++ b/pulsaradmin/pkg/utils/home_dir.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/inactive_topic_policies.go b/pulsaradmin/pkg/utils/inactive_topic_policies.go index fdff72d1ef..05f81b6645 100644 --- a/pulsaradmin/pkg/utils/inactive_topic_policies.go +++ b/pulsaradmin/pkg/utils/inactive_topic_policies.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/internal_configuration_data.go b/pulsaradmin/pkg/utils/internal_configuration_data.go index 6bcd6f2ff2..75cad0e245 100644 --- a/pulsaradmin/pkg/utils/internal_configuration_data.go +++ b/pulsaradmin/pkg/utils/internal_configuration_data.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/load_manage_report.go b/pulsaradmin/pkg/utils/load_manage_report.go index d52e2342a4..5196da9b55 100644 --- a/pulsaradmin/pkg/utils/load_manage_report.go +++ b/pulsaradmin/pkg/utils/load_manage_report.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/long_running_process_status.go b/pulsaradmin/pkg/utils/long_running_process_status.go index 37b57aa60b..c61919ecc8 100644 --- a/pulsaradmin/pkg/utils/long_running_process_status.go +++ b/pulsaradmin/pkg/utils/long_running_process_status.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/message.go b/pulsaradmin/pkg/utils/message.go index 29b00b6cb0..2f0e4befe8 100644 --- a/pulsaradmin/pkg/utils/message.go +++ b/pulsaradmin/pkg/utils/message.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/message_id.go b/pulsaradmin/pkg/utils/message_id.go index 014de3a050..d75b613e1b 100644 --- a/pulsaradmin/pkg/utils/message_id.go +++ b/pulsaradmin/pkg/utils/message_id.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/message_id_test.go b/pulsaradmin/pkg/utils/message_id_test.go index 9c74befaaa..1ce0b5230b 100644 --- a/pulsaradmin/pkg/utils/message_id_test.go +++ b/pulsaradmin/pkg/utils/message_id_test.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/metrics.go b/pulsaradmin/pkg/utils/metrics.go index b42f5a8c72..ca1919423d 100644 --- a/pulsaradmin/pkg/utils/metrics.go +++ b/pulsaradmin/pkg/utils/metrics.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/namespace_name.go b/pulsaradmin/pkg/utils/namespace_name.go index 3fbab820eb..11b7435545 100644 --- a/pulsaradmin/pkg/utils/namespace_name.go +++ b/pulsaradmin/pkg/utils/namespace_name.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/namespace_name_test.go b/pulsaradmin/pkg/utils/namespace_name_test.go index 96505dba32..bc12070e9a 100644 --- a/pulsaradmin/pkg/utils/namespace_name_test.go +++ b/pulsaradmin/pkg/utils/namespace_name_test.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/ns_isolation_data.go b/pulsaradmin/pkg/utils/ns_isolation_data.go index 7fefad4ea6..4589eb30c3 100644 --- a/pulsaradmin/pkg/utils/ns_isolation_data.go +++ b/pulsaradmin/pkg/utils/ns_isolation_data.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/ns_ownership_status.go b/pulsaradmin/pkg/utils/ns_ownership_status.go index dbae8d379b..bb63b704f8 100644 --- a/pulsaradmin/pkg/utils/ns_ownership_status.go +++ b/pulsaradmin/pkg/utils/ns_ownership_status.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/package_metadata.go b/pulsaradmin/pkg/utils/package_metadata.go index bb816e8d92..860afd9c8e 100644 --- a/pulsaradmin/pkg/utils/package_metadata.go +++ b/pulsaradmin/pkg/utils/package_metadata.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/package_name.go b/pulsaradmin/pkg/utils/package_name.go index 0492da0d60..afc646d687 100644 --- a/pulsaradmin/pkg/utils/package_name.go +++ b/pulsaradmin/pkg/utils/package_name.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/package_name_test.go b/pulsaradmin/pkg/utils/package_name_test.go index 97f8917427..8e6edd52d8 100644 --- a/pulsaradmin/pkg/utils/package_name_test.go +++ b/pulsaradmin/pkg/utils/package_name_test.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/package_type.go b/pulsaradmin/pkg/utils/package_type.go index c7af374f02..70f455b253 100644 --- a/pulsaradmin/pkg/utils/package_type.go +++ b/pulsaradmin/pkg/utils/package_type.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/persistence_policies.go b/pulsaradmin/pkg/utils/persistence_policies.go index 4eec771185..d4c8bdb04e 100644 --- a/pulsaradmin/pkg/utils/persistence_policies.go +++ b/pulsaradmin/pkg/utils/persistence_policies.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/policies.go b/pulsaradmin/pkg/utils/policies.go index bfb7ca05a7..3d727994c3 100644 --- a/pulsaradmin/pkg/utils/policies.go +++ b/pulsaradmin/pkg/utils/policies.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/producer_config.go b/pulsaradmin/pkg/utils/producer_config.go index 591fa01257..d4c1c3eb6d 100644 --- a/pulsaradmin/pkg/utils/producer_config.go +++ b/pulsaradmin/pkg/utils/producer_config.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/publish_rate.go b/pulsaradmin/pkg/utils/publish_rate.go index 04e0335d55..9dbc2d00d2 100644 --- a/pulsaradmin/pkg/utils/publish_rate.go +++ b/pulsaradmin/pkg/utils/publish_rate.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/resource_quota.go b/pulsaradmin/pkg/utils/resource_quota.go index ccb11b11a0..d9ef4f74db 100644 --- a/pulsaradmin/pkg/utils/resource_quota.go +++ b/pulsaradmin/pkg/utils/resource_quota.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/resources.go b/pulsaradmin/pkg/utils/resources.go index 6212750013..a4f3ddbcbe 100644 --- a/pulsaradmin/pkg/utils/resources.go +++ b/pulsaradmin/pkg/utils/resources.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/retention_policies.go b/pulsaradmin/pkg/utils/retention_policies.go index e0cdb9d096..55bf915ed2 100644 --- a/pulsaradmin/pkg/utils/retention_policies.go +++ b/pulsaradmin/pkg/utils/retention_policies.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/schema_strategy.go b/pulsaradmin/pkg/utils/schema_strategy.go index 9dfa1c6f7f..176f0e0a90 100644 --- a/pulsaradmin/pkg/utils/schema_strategy.go +++ b/pulsaradmin/pkg/utils/schema_strategy.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/schema_util.go b/pulsaradmin/pkg/utils/schema_util.go index 52c8f0c059..08aaf54ac6 100644 --- a/pulsaradmin/pkg/utils/schema_util.go +++ b/pulsaradmin/pkg/utils/schema_util.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/sink_config.go b/pulsaradmin/pkg/utils/sink_config.go index b0c5eb1bf2..0e9163bf91 100644 --- a/pulsaradmin/pkg/utils/sink_config.go +++ b/pulsaradmin/pkg/utils/sink_config.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/sink_status.go b/pulsaradmin/pkg/utils/sink_status.go index 7aae14475f..6cdb091fac 100644 --- a/pulsaradmin/pkg/utils/sink_status.go +++ b/pulsaradmin/pkg/utils/sink_status.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/source_config.go b/pulsaradmin/pkg/utils/source_config.go index 506d3eb260..7b0747610f 100644 --- a/pulsaradmin/pkg/utils/source_config.go +++ b/pulsaradmin/pkg/utils/source_config.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/source_status.go b/pulsaradmin/pkg/utils/source_status.go index ffb617419c..71df5a4fa8 100644 --- a/pulsaradmin/pkg/utils/source_status.go +++ b/pulsaradmin/pkg/utils/source_status.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/subscription_auth_mode.go b/pulsaradmin/pkg/utils/subscription_auth_mode.go index d2b57885cb..795b6d0aea 100644 --- a/pulsaradmin/pkg/utils/subscription_auth_mode.go +++ b/pulsaradmin/pkg/utils/subscription_auth_mode.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/topic_auto_creation_config.go b/pulsaradmin/pkg/utils/topic_auto_creation_config.go index d4413e1e77..6664655974 100644 --- a/pulsaradmin/pkg/utils/topic_auto_creation_config.go +++ b/pulsaradmin/pkg/utils/topic_auto_creation_config.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/topic_domain.go b/pulsaradmin/pkg/utils/topic_domain.go index 3f7859f637..98c59a9cec 100644 --- a/pulsaradmin/pkg/utils/topic_domain.go +++ b/pulsaradmin/pkg/utils/topic_domain.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/topic_name.go b/pulsaradmin/pkg/utils/topic_name.go index 324ac42111..268abd73d1 100644 --- a/pulsaradmin/pkg/utils/topic_name.go +++ b/pulsaradmin/pkg/utils/topic_name.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/topic_name_test.go b/pulsaradmin/pkg/utils/topic_name_test.go index b6bdc114a2..27e5002488 100644 --- a/pulsaradmin/pkg/utils/topic_name_test.go +++ b/pulsaradmin/pkg/utils/topic_name_test.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/topic_type.go b/pulsaradmin/pkg/utils/topic_type.go index f3723b9799..18320594ac 100644 --- a/pulsaradmin/pkg/utils/topic_type.go +++ b/pulsaradmin/pkg/utils/topic_type.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/topics_stats_stream.go b/pulsaradmin/pkg/utils/topics_stats_stream.go index bf21e41fa8..7554609e6e 100644 --- a/pulsaradmin/pkg/utils/topics_stats_stream.go +++ b/pulsaradmin/pkg/utils/topics_stats_stream.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/update_options.go b/pulsaradmin/pkg/utils/update_options.go index 0ff183e422..d78fccfcec 100644 --- a/pulsaradmin/pkg/utils/update_options.go +++ b/pulsaradmin/pkg/utils/update_options.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/utils.go b/pulsaradmin/pkg/utils/utils.go index c1d5143cbb..7b23b15654 100644 --- a/pulsaradmin/pkg/utils/utils.go +++ b/pulsaradmin/pkg/utils/utils.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/utils_test.go b/pulsaradmin/pkg/utils/utils_test.go index 5c3ddaac15..7a7fe4abe3 100644 --- a/pulsaradmin/pkg/utils/utils_test.go +++ b/pulsaradmin/pkg/utils/utils_test.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/window_confing.go b/pulsaradmin/pkg/utils/window_confing.go index a157532db7..ccde3b68fc 100644 --- a/pulsaradmin/pkg/utils/window_confing.go +++ b/pulsaradmin/pkg/utils/window_confing.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // diff --git a/pulsaradmin/pkg/utils/worker_info.go b/pulsaradmin/pkg/utils/worker_info.go index 947e447e40..bd0dd806c0 100644 --- a/pulsaradmin/pkg/utils/worker_info.go +++ b/pulsaradmin/pkg/utils/worker_info.go @@ -1,8 +1,10 @@ -// Copyright 2023 StreamNative, Inc. -// -// 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 +// 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 // From fbee610ddcbf126ec93ee7d78c3077c8a1672351 Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 16 Aug 2023 16:18:03 +0800 Subject: [PATCH 271/348] revrert: allow rebase and merge (#1081) Reverts #1080 The repo merge has been done. Regular PRs should go with squash and merge. --- .asf.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index ddaa0df7d0..246ec81885 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -40,7 +40,7 @@ github: # disable merge button: merge: false # disable rebase button: - rebase: true + rebase: false notifications: From 41093516af21cf2238e11532d877e1aab73f95bf Mon Sep 17 00:00:00 2001 From: grayson <916028390@qq.com> Date: Wed, 30 Aug 2023 15:26:38 +0800 Subject: [PATCH 272/348] fix(producer): fail all messages that are pending requests when closing (#1059) This aligns the manner with Java client. --- pulsar/producer_partition.go | 60 ++++++++++++++++++++++++++++++++++++ pulsar/producer_test.go | 30 ++++++++++++++++++ 2 files changed, 90 insertions(+) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index d3f61ef101..720c7df485 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1291,6 +1291,8 @@ func (p *partitionProducer) internalClose(req *closeProducer) { return } + defer close(p.dataChan) + defer close(p.cmdChan) p.log.Info("Closing producer") id := p.client.rpcClient.NewRequestID() @@ -1304,6 +1306,7 @@ func (p *partitionProducer) internalClose(req *closeProducer) { } else { p.log.Info("Closed producer") } + p.failPendingMessages() if p.batchBuilder != nil { if err = p.batchBuilder.Close(); err != nil { @@ -1316,6 +1319,63 @@ func (p *partitionProducer) internalClose(req *closeProducer) { p.batchFlushTicker.Stop() } +func (p *partitionProducer) failPendingMessages() { + curViewItems := p.pendingQueue.ReadableSlice() + viewSize := len(curViewItems) + if viewSize <= 0 { + return + } + p.log.Infof("Failing %d messages on closing producer", viewSize) + lastViewItem := curViewItems[viewSize-1].(*pendingItem) + + // iterate at most viewSize items + for i := 0; i < viewSize; i++ { + item := p.pendingQueue.CompareAndPoll( + func(m interface{}) bool { + return m != nil + }) + + if item == nil { + return + } + + pi := item.(*pendingItem) + pi.Lock() + + for _, i := range pi.sendRequests { + sr := i.(*sendRequest) + if sr.msg != nil { + size := len(sr.msg.Payload) + p.releaseSemaphoreAndMem(sr.reservedMem) + p.metrics.MessagesPending.Dec() + p.metrics.BytesPending.Sub(float64(size)) + p.log.WithError(errProducerClosed). + WithField("size", size). + WithField("properties", sr.msg.Properties) + } + + if sr.callback != nil { + sr.callbackOnce.Do(func() { + runCallback(sr.callback, nil, sr.msg, errProducerClosed) + }) + } + if sr.transaction != nil { + sr.transaction.endSendOrAckOp(nil) + } + } + + // flag the sending has completed with error, flush make no effect + pi.Complete(errProducerClosed) + pi.Unlock() + + // finally reached the last view item, current iteration ends + if pi == lastViewItem { + p.log.Infof("%d messages complete failed", viewSize) + return + } + } +} + func (p *partitionProducer) LastSequenceID() int64 { return atomic.LoadInt64(&p.lastSequenceID) } diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index a9a7b81911..38fec57680 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -2268,3 +2268,33 @@ func TestProducerSendWithContext(t *testing.T) { // producer.Send should fail and return err context.Canceled assert.True(t, errors.Is(err, context.Canceled)) } + +func TestFailPendingMessageWithClose(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + assert.NoError(t, err) + defer client.Close() + testProducer, err := client.CreateProducer(ProducerOptions{ + Topic: newTopicName(), + DisableBlockIfQueueFull: false, + BatchingMaxPublishDelay: 100000, + BatchingMaxMessages: 1000, + }) + + assert.NoError(t, err) + assert.NotNil(t, testProducer) + for i := 0; i < 3; i++ { + testProducer.SendAsync(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }, func(id MessageID, message *ProducerMessage, e error) { + if e != nil { + assert.Equal(t, errProducerClosed, e) + } + }) + } + partitionProducerImp := testProducer.(*producer).producers[0].(*partitionProducer) + partitionProducerImp.pendingQueue.Put(&pendingItem{}) + testProducer.Close() + assert.Equal(t, 0, partitionProducerImp.pendingQueue.Size()) +} From 28a6b49159a7474facdb6dc5d675fae427d8a1f8 Mon Sep 17 00:00:00 2001 From: Guangning E Date: Thu, 7 Sep 2023 20:38:55 +0800 Subject: [PATCH 273/348] Support load config from env (#1089) * Support load config from env * Run goimport --- pulsaradmin/pkg/admin/auth/oauth2.go | 8 ++++++-- pulsaradmin/pkg/utils/utils.go | 10 ++++++++++ 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/pkg/admin/auth/oauth2.go b/pulsaradmin/pkg/admin/auth/oauth2.go index eab64cace5..2864eae2dd 100644 --- a/pulsaradmin/pkg/admin/auth/oauth2.go +++ b/pulsaradmin/pkg/admin/auth/oauth2.go @@ -19,9 +19,12 @@ package auth import ( "encoding/json" + "fmt" "net/http" "path/filepath" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" + "github.com/99designs/keyring" "github.com/apache/pulsar-client-go/oauth2" "github.com/apache/pulsar-client-go/oauth2/cache" @@ -243,8 +246,9 @@ func makeKeyring() (keyring.Keyring, error) { ServiceName: serviceName, KeychainName: keyChainName, KeychainTrustApplication: true, - FileDir: filepath.Join("~/.config/pulsar", "credentials"), - FilePasswordFunc: keyringPrompt, + FileDir: filepath.Join(fmt.Sprintf( + "%s/.config/pulsar", utils.GetConfigPath()), "credentials"), + FilePasswordFunc: keyringPrompt, }) } diff --git a/pulsaradmin/pkg/utils/utils.go b/pulsaradmin/pkg/utils/utils.go index 7b23b15654..fc0897cab1 100644 --- a/pulsaradmin/pkg/utils/utils.go +++ b/pulsaradmin/pkg/utils/utils.go @@ -19,6 +19,7 @@ package utils import ( "fmt" + "os" "reflect" ) @@ -36,3 +37,12 @@ func IsNilFixed(i interface{}) bool { } return false } + +func GetConfigPath() string { + // pulsar client config dir path, for example: PULSAR_CLIENT_CONF_PATH: /home/pulsar + // configuration file path is: /home/pulsar/.config/pulsar + if envConf, ok := os.LookupEnv("PULSAR_CLIENT_CONF_PATH"); ok { + return envConf + } + return HomeDir() +} From 2a15a251d25bd2a6276051d71873e9e106cc1e85 Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Sun, 10 Sep 2023 11:43:30 +0800 Subject: [PATCH 274/348] [Fix] Multiple calls to client.Close causes panic (#1046) --- pulsar/client_impl.go | 10 +++++++--- pulsar/client_impl_test.go | 7 +++++++ 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index c283f5a827..801eab3f28 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -20,6 +20,7 @@ package pulsar import ( "fmt" "net/url" + "sync" "time" "github.com/apache/pulsar-client-go/pulsar/auth" @@ -47,6 +48,7 @@ type client struct { metrics *internal.Metrics tcClient *transactionCoordinatorClient memLimit internal.MemoryLimitController + closeOnce sync.Once log log.Logger } @@ -266,7 +268,9 @@ func (c *client) TopicPartitions(topic string) ([]string, error) { } func (c *client) Close() { - c.handlers.Close() - c.cnxPool.Close() - c.lookupService.Close() + c.closeOnce.Do(func() { + c.handlers.Close() + c.cnxPool.Close() + c.lookupService.Close() + }) } diff --git a/pulsar/client_impl_test.go b/pulsar/client_impl_test.go index bb28371f20..78dc1cae68 100644 --- a/pulsar/client_impl_test.go +++ b/pulsar/client_impl_test.go @@ -1238,3 +1238,10 @@ func TestAutoCloseIdleConnection(t *testing.T) { cli.Close() } + +func TestMultipleCloseClient(t *testing.T) { + client, err := NewClient(ClientOptions{URL: serviceURL}) + assert.Nil(t, err) + client.Close() + client.Close() +} From 7fef6a985b2a880bd47948afe0bc546ffc0c958d Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Wed, 13 Sep 2023 14:52:40 +0800 Subject: [PATCH 275/348] [improve][client] Implement GetLastMSgID for Reader (#1087) ### Motivation Implement the GetLastMSgID API for Reader. --------- Co-authored-by: Yunze Xu --- pulsar/reader.go | 3 +++ pulsar/reader_impl.go | 4 ++++ pulsar/reader_test.go | 42 ++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 49 insertions(+) diff --git a/pulsar/reader.go b/pulsar/reader.go index d58d06f6fe..5e1a73b988 100644 --- a/pulsar/reader.go +++ b/pulsar/reader.go @@ -134,4 +134,7 @@ type Reader interface { // the message publish time where to reposition the subscription // SeekByTime(time time.Time) error + + // GetLastMessageID get the last message id available for consume. + GetLastMessageID() (MessageID, error) } diff --git a/pulsar/reader_impl.go b/pulsar/reader_impl.go index 5a2128a377..ffc92dedde 100644 --- a/pulsar/reader_impl.go +++ b/pulsar/reader_impl.go @@ -244,3 +244,7 @@ func (r *reader) SeekByTime(time time.Time) error { return r.pc.SeekByTime(time) } + +func (r *reader) GetLastMessageID() (MessageID, error) { + return r.pc.getLastMessageID() +} diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index 0a1b2a1df4..ec10f8f162 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -901,3 +901,45 @@ func TestReaderWithBackoffPolicy(t *testing.T) { partitionConsumerImp.reconnectToBroker() assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) } + +func TestReaderGetLastMessageID(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: serviceURL, + }) + assert.Nil(t, err) + topic := newTopicName() + ctx := context.Background() + schema := NewStringSchema(nil) + // create producer + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + DisableBatching: true, + Schema: schema, + }) + assert.Nil(t, err) + defer producer.Close() + + var lastMsgID MessageID + // send 10 messages + for i := 0; i < 10; i++ { + msgID, err := producer.Send(ctx, &ProducerMessage{ + Payload: []byte(fmt.Sprintf("hello-%d", i)), + }) + assert.NoError(t, err) + assert.NotNil(t, msgID) + lastMsgID = msgID + } + + reader, err := client.CreateReader(ReaderOptions{ + Topic: topic, + StartMessageID: EarliestMessageID(), + }) + assert.Nil(t, err) + getLastMessageID, err := reader.GetLastMessageID() + if err != nil { + return + } + + assert.Equal(t, lastMsgID.LedgerID(), getLastMessageID.LedgerID()) + assert.Equal(t, lastMsgID.EntryID(), getLastMessageID.EntryID()) +} From f979c183c55b6efdceaa1cebcb62f1c20f8ef604 Mon Sep 17 00:00:00 2001 From: Masahiro Sakamoto Date: Wed, 13 Sep 2023 20:58:42 +0900 Subject: [PATCH 276/348] [fix] Fix comment for ConnectionMaxIdleTime (#1091) ### Motivation The default value of `ConnectionMaxIdleTime` is said to be 60 seconds in the comment, /~https://github.com/apache/pulsar-client-go/blob/2a15a251d25bd2a6276051d71873e9e106cc1e85/pulsar/client.go#L155-L157 but it actually seems to be 180 seconds. 60 seconds is the minimum value, not the default value. /~https://github.com/apache/pulsar-client-go/blob/2a15a251d25bd2a6276051d71873e9e106cc1e85/pulsar/client_impl.go#L39-L40 --- pulsar/client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar/client.go b/pulsar/client.go index d6f18c9aa9..d9ac2a73a2 100644 --- a/pulsar/client.go +++ b/pulsar/client.go @@ -153,7 +153,7 @@ type ClientOptions struct { MetricsRegisterer prometheus.Registerer // Release the connection if it is not used for more than ConnectionMaxIdleTime. - // Default is 60 seconds, negative such as -1 to disable. + // Default is 180 seconds, minimum is 60 seconds. Negative such as -1 to disable. ConnectionMaxIdleTime time.Duration EnableTransaction bool From 7cf643be20b0eed9b37e168d04884e89f534efdd Mon Sep 17 00:00:00 2001 From: ming Date: Fri, 15 Sep 2023 11:32:28 -0400 Subject: [PATCH 277/348] connectionTimeout respects net.Dialer default timeout (#1095) --- pulsar/client_impl.go | 7 +++---- pulsar/internal/connection.go | 8 +++++++- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index 801eab3f28..7daf6f62ab 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -31,7 +31,6 @@ import ( ) const ( - defaultConnectionTimeout = 10 * time.Second defaultOperationTimeout = 30 * time.Second defaultKeepAliveInterval = 30 * time.Second defaultMemoryLimitBytes = 64 * 1024 * 1024 @@ -117,10 +116,10 @@ func newClient(options ClientOptions) (Client, error) { return nil, err } + // the default timeout respects Go's default timeout which is no timeout + // Missing user specified timeout renders 0 values that matches + // net.Dailer's default if time.Duration value is not initialized connectionTimeout := options.ConnectionTimeout - if connectionTimeout.Nanoseconds() == 0 { - connectionTimeout = defaultConnectionTimeout - } operationTimeout := options.OperationTimeout if operationTimeout.Nanoseconds() == 0 { diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index e2ae7ac845..840ecc4f41 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -256,7 +256,11 @@ func (c *connection) connect() bool { if c.tlsOptions == nil { // Clear text connection - cnx, err = net.DialTimeout("tcp", c.physicalAddr.Host, c.connectionTimeout) + if c.connectionTimeout.Nanoseconds() > 0 { + cnx, err = net.DialTimeout("tcp", c.physicalAddr.Host, c.connectionTimeout) + } else { + cnx, err = net.Dial("tcp", c.physicalAddr.Host) + } } else { // TLS connection tlsConfig, err = c.getTLSConfig() @@ -265,6 +269,8 @@ func (c *connection) connect() bool { return false } + // time.Duration is initialized to 0 by default, net.Dialer's default timeout is no timeout + // therefore if c.connectionTimeout is 0, it means no timeout d := &net.Dialer{Timeout: c.connectionTimeout} cnx, err = tls.DialWithDialer(d, "tcp", c.physicalAddr.Host, tlsConfig) } From ac9c1a6399336461d2d3ce1cdd31cac6debd5ed5 Mon Sep 17 00:00:00 2001 From: Donglai Fu Date: Mon, 25 Sep 2023 09:38:39 +0800 Subject: [PATCH 278/348] Support OAuth2 with scope field (#1097) ### Motivation Current `pulsaradmin` OAuth2 params don't support to input the scope field. Add a function to implement it. `ClientCredentialsFlowOptions` has a `AdditionalScopes` field to set scopes. --- pulsaradmin/pkg/admin/auth/oauth2.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/pkg/admin/auth/oauth2.go b/pulsaradmin/pkg/admin/auth/oauth2.go index 2864eae2dd..4bd1546195 100644 --- a/pulsaradmin/pkg/admin/auth/oauth2.go +++ b/pulsaradmin/pkg/admin/auth/oauth2.go @@ -73,10 +73,15 @@ func NewAuthenticationOAuth2(issuer oauth2.Issuer, store store.Store) (*OAuth2Pr // NewAuthenticationOAuth2WithDefaultFlow uses memory to save the grant func NewAuthenticationOAuth2WithDefaultFlow(issuer oauth2.Issuer, keyFile string) (Provider, error) { - st := store.NewMemoryStore() - flow, err := oauth2.NewDefaultClientCredentialsFlow(oauth2.ClientCredentialsFlowOptions{ + return NewAuthenticationOAuth2WithFlow(issuer, oauth2.ClientCredentialsFlowOptions{ KeyFile: keyFile, }) +} + +func NewAuthenticationOAuth2WithFlow( + issuer oauth2.Issuer, flowOptions oauth2.ClientCredentialsFlowOptions) (Provider, error) { + st := store.NewMemoryStore() + flow, err := oauth2.NewDefaultClientCredentialsFlow(flowOptions) if err != nil { return nil, err } From 0bf2f9b86f4e540e912f73c65327b9dc6c4b57c9 Mon Sep 17 00:00:00 2001 From: Masahiro Sakamoto Date: Tue, 10 Oct 2023 11:49:26 +0900 Subject: [PATCH 279/348] Fix issue where DisableReplication flag does not work (#1100) --- pulsar/producer_partition.go | 4 +- pulsar/producer_test.go | 92 ++++++++++++++++++++++++++++++++++++ 2 files changed, 94 insertions(+), 2 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 720c7df485..ebd292ec11 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -551,13 +551,13 @@ func (p *partitionProducer) internalSend(request *sendRequest) { deliverAt = time.Now().Add(msg.DeliverAfter) } - mm := p.genMetadata(msg, uncompressedSize, deliverAt) - // set default ReplicationClusters when DisableReplication if msg.DisableReplication { msg.ReplicationClusters = []string{"__local__"} } + mm := p.genMetadata(msg, uncompressedSize, deliverAt) + sendAsBatch := !p.options.DisableBatching && msg.ReplicationClusters == nil && deliverAt.UnixNano() < 0 diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 38fec57680..29ffa7805a 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -30,7 +30,9 @@ import ( "time" "github.com/apache/pulsar-client-go/pulsar/internal" + pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" "github.com/stretchr/testify/assert" + "google.golang.org/protobuf/proto" "github.com/apache/pulsar-client-go/pulsar/crypto" plog "github.com/apache/pulsar-client-go/pulsar/log" @@ -2298,3 +2300,93 @@ func TestFailPendingMessageWithClose(t *testing.T) { testProducer.Close() assert.Equal(t, 0, partitionProducerImp.pendingQueue.Size()) } + +type pendingQueueWrapper struct { + pendingQueue internal.BlockingQueue + writtenBuffers *[]internal.Buffer +} + +func (pqw *pendingQueueWrapper) Put(item interface{}) { + pi := item.(*pendingItem) + writerIdx := pi.buffer.WriterIndex() + buf := internal.NewBuffer(int(writerIdx)) + buf.Write(pi.buffer.Get(0, writerIdx)) + *pqw.writtenBuffers = append(*pqw.writtenBuffers, buf) + pqw.pendingQueue.Put(item) +} + +func (pqw *pendingQueueWrapper) Take() interface{} { + return pqw.pendingQueue.Take() +} + +func (pqw *pendingQueueWrapper) Poll() interface{} { + return pqw.pendingQueue.Poll() +} + +func (pqw *pendingQueueWrapper) CompareAndPoll(compare func(interface{}) bool) interface{} { + return pqw.pendingQueue.CompareAndPoll(compare) +} + +func (pqw *pendingQueueWrapper) Peek() interface{} { + return pqw.pendingQueue.Peek() +} + +func (pqw *pendingQueueWrapper) PeekLast() interface{} { + return pqw.pendingQueue.PeekLast() +} + +func (pqw *pendingQueueWrapper) Size() int { + return pqw.pendingQueue.Size() +} + +func (pqw *pendingQueueWrapper) ReadableSlice() []interface{} { + return pqw.pendingQueue.ReadableSlice() +} + +func TestDisableReplication(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: serviceURL, + }) + assert.NoError(t, err) + defer client.Close() + + testProducer, err := client.CreateProducer(ProducerOptions{ + Topic: newTopicName(), + DisableBatching: true, + }) + assert.NoError(t, err) + assert.NotNil(t, testProducer) + defer testProducer.Close() + + writtenBuffers := make([]internal.Buffer, 0) + pqw := &pendingQueueWrapper{ + pendingQueue: internal.NewBlockingQueue(1000), + writtenBuffers: &writtenBuffers, + } + + partitionProducerImp := testProducer.(*producer).producers[0].(*partitionProducer) + partitionProducerImp.pendingQueue = pqw + + ID, err := testProducer.Send(context.Background(), &ProducerMessage{ + Payload: []byte("disable-replication"), + DisableReplication: true, + }) + assert.NoError(t, err) + assert.NotNil(t, ID) + + assert.Equal(t, 1, len(writtenBuffers)) + buf := writtenBuffers[0] + + buf.Skip(4) // TOTAL_SIZE + cmdSize := buf.ReadUint32() // CMD_SIZE + buf.Skip(cmdSize) // CMD + buf.Skip(2) // MAGIC_NUMBER + buf.Skip(4) // CHECKSUM + metadataSize := buf.ReadUint32() // METADATA_SIZE + metadata := buf.Read(metadataSize) // METADATA + + var msgMetadata pb.MessageMetadata + err = proto.Unmarshal(metadata, &msgMetadata) + assert.NoError(t, err) + assert.Equal(t, []string{"__local__"}, msgMetadata.GetReplicateTo()) +} From da06dca2935c087c51365c846ed2116a5ac0d653 Mon Sep 17 00:00:00 2001 From: zccold Date: Sat, 21 Oct 2023 19:56:17 +0800 Subject: [PATCH 280/348] dobule check before consumer reconnect (#1084) --- pulsar/consumer_partition.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 364dae754c..2ba842744c 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -1644,6 +1644,13 @@ func (pc *partitionConsumer) reconnectToBroker() { pc.log.Info("Reconnecting to broker in ", delayReconnectTime) time.Sleep(delayReconnectTime) + // double check + if pc.getConsumerState() != consumerReady { + // Consumer is already closing + pc.log.Info("consumer state not ready, exit reconnect") + return + } + err := pc.grabConn() if err == nil { // Successfully reconnected From f5667b249a7740764cd5046045bf3dab8341d389 Mon Sep 17 00:00:00 2001 From: Lei Zhiyuan Date: Mon, 23 Oct 2023 22:00:58 +0800 Subject: [PATCH 281/348] fix: fix schema error (#823) Co-authored-by: zhiyuanlei --- pulsar/schema.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar/schema.go b/pulsar/schema.go index 5c063e32a3..fd9d412dc6 100644 --- a/pulsar/schema.go +++ b/pulsar/schema.go @@ -54,7 +54,7 @@ const ( _ // _ // KeyValue //A Schema that contains Key Schema and Value Schema. - BYTES = -1 //A bytes array. + BYTES = 0 //A bytes array. AUTO = -2 // AutoConsume = -3 //Auto Consume Type. AutoPublish = -4 // Auto Publish Type. From 59c7eaee9644bf99eee1b5e82540a847466ee8d4 Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Tue, 24 Oct 2023 14:32:18 +0800 Subject: [PATCH 282/348] refactor: rename pendingItem.Complete() to pendingItem.done() (#1109) Co-authored-by: gunli --- pulsar/producer_partition.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index ebd292ec11..82123f988a 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -862,7 +862,7 @@ type pendingItem struct { sequenceID uint64 sentAt time.Time sendRequests []interface{} - completed bool + isDone bool flushCallback func(err error) } @@ -1001,7 +1001,7 @@ func (p *partitionProducer) failTimeoutMessages() { } // flag the sending has completed with error, flush make no effect - pi.Complete(errSendTimeout) + pi.done(errSendTimeout) pi.Unlock() // finally reached the last view item, current iteration ends @@ -1076,7 +1076,7 @@ func (p *partitionProducer) internalFlush(fr *flushRequest) { pi.Lock() defer pi.Unlock() - if pi.completed { + if pi.isDone { // The last item in the queue has been completed while we were // looking at it. It's safe at this point to assume that every // message enqueued before Flush() was called are now persisted @@ -1281,7 +1281,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) } // Mark this pending item as done - pi.Complete(nil) + pi.done(nil) } } @@ -1365,7 +1365,7 @@ func (p *partitionProducer) failPendingMessages() { } // flag the sending has completed with error, flush make no effect - pi.Complete(errProducerClosed) + pi.done(errProducerClosed) pi.Unlock() // finally reached the last view item, current iteration ends @@ -1452,11 +1452,11 @@ type flushRequest struct { err error } -func (i *pendingItem) Complete(err error) { - if i.completed { +func (i *pendingItem) done(err error) { + if i.isDone { return } - i.completed = true + i.isDone = true buffersPool.Put(i.buffer) if i.flushCallback != nil { i.flushCallback(err) From f9969ca84e927786ef208abb73de065eb6bfacae Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Tue, 24 Oct 2023 16:42:31 +0800 Subject: [PATCH 283/348] feat: add sendRequest.done() to release resource together (#1110) Co-authored-by: gunli --- pulsar/producer_partition.go | 93 ++++++++++++++++++++++++++++++------ 1 file changed, 79 insertions(+), 14 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 82123f988a..2beb36144a 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1419,21 +1419,37 @@ func (p *partitionProducer) Close() { <-cp.doneCh } +//nolint:all type sendRequest struct { - ctx context.Context - msg *ProducerMessage - callback func(MessageID, *ProducerMessage, error) - callbackOnce *sync.Once - publishTime time.Time - flushImmediately bool - blockCh chan struct{} - closeBlockChOnce *sync.Once - totalChunks int - chunkID int - uuid string - chunkRecorder *chunkRecorder - transaction *transaction - reservedMem int64 + ctx context.Context + msg *ProducerMessage + callback func(MessageID, *ProducerMessage, error) + callbackOnce *sync.Once + publishTime time.Time + flushImmediately bool + blockCh chan struct{} + closeBlockChOnce *sync.Once + totalChunks int + chunkID int + uuid string + chunkRecorder *chunkRecorder + transaction *transaction + reservedMem int64 + producer *partitionProducer + memLimit internal.MemoryLimitController + semaphore internal.Semaphore + reservedSemaphore int + sendAsBatch bool + schema Schema + schemaVersion []byte + uncompressedPayload []byte + uncompressedSize int64 + compressedPayload []byte + compressedSize int + payloadChunkSize int + mm *pb.MessageMetadata + deliverAt time.Time + maxMessageSize int32 } // stopBlock can be invoked multiple times safety @@ -1443,6 +1459,55 @@ func (sr *sendRequest) stopBlock() { }) } +//nolint:all +func (sr *sendRequest) done(msgID MessageID, err error) { + if err == nil { + sr.producer.metrics.PublishLatency.Observe(float64(time.Now().UnixNano()-sr.publishTime.UnixNano()) / 1.0e9) + sr.producer.metrics.MessagesPublished.Inc() + sr.producer.metrics.BytesPublished.Add(float64(sr.reservedMem)) + } + + if err != nil { + sr.producer.log.WithError(err). + WithField("size", sr.reservedMem). + WithField("properties", sr.msg.Properties) + } + + if err == errSendTimeout { + sr.producer.metrics.PublishErrorsTimeout.Inc() + } + + if err == errMessageTooLarge { + sr.producer.metrics.PublishErrorsMsgTooLarge.Inc() + } + + if sr.semaphore != nil { + for i := 0; i < sr.reservedSemaphore; i++ { + sr.semaphore.Release() + } + sr.producer.metrics.MessagesPending.Sub(float64(sr.reservedSemaphore)) + } + + if sr.memLimit != nil { + sr.memLimit.ReleaseMemory(sr.reservedMem) + sr.producer.metrics.BytesPending.Sub(float64(sr.reservedMem)) + } + + if sr.totalChunks <= 1 || sr.chunkID == sr.totalChunks-1 { + sr.callbackOnce.Do(func() { + runCallback(sr.callback, msgID, sr.msg, err) + }) + + if sr.transaction != nil { + sr.transaction.endSendOrAckOp(err) + } + + if sr.producer.options.Interceptors != nil && err == nil { + sr.producer.options.Interceptors.OnSendAcknowledgement(sr.producer, sr.msg, msgID) + } + } +} + type closeProducer struct { doneCh chan struct{} } From e61e96662cfc850bdda48a7ff70541c81ddf0cde Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 24 Oct 2023 19:29:32 +0800 Subject: [PATCH 284/348] Revert "feat: add sendRequest.done() to release resource together (#1110)" This needs a rework. This reverts commit f9969ca84e927786ef208abb73de065eb6bfacae. --- pulsar/producer_partition.go | 93 ++++++------------------------------ 1 file changed, 14 insertions(+), 79 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 2beb36144a..82123f988a 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1419,37 +1419,21 @@ func (p *partitionProducer) Close() { <-cp.doneCh } -//nolint:all type sendRequest struct { - ctx context.Context - msg *ProducerMessage - callback func(MessageID, *ProducerMessage, error) - callbackOnce *sync.Once - publishTime time.Time - flushImmediately bool - blockCh chan struct{} - closeBlockChOnce *sync.Once - totalChunks int - chunkID int - uuid string - chunkRecorder *chunkRecorder - transaction *transaction - reservedMem int64 - producer *partitionProducer - memLimit internal.MemoryLimitController - semaphore internal.Semaphore - reservedSemaphore int - sendAsBatch bool - schema Schema - schemaVersion []byte - uncompressedPayload []byte - uncompressedSize int64 - compressedPayload []byte - compressedSize int - payloadChunkSize int - mm *pb.MessageMetadata - deliverAt time.Time - maxMessageSize int32 + ctx context.Context + msg *ProducerMessage + callback func(MessageID, *ProducerMessage, error) + callbackOnce *sync.Once + publishTime time.Time + flushImmediately bool + blockCh chan struct{} + closeBlockChOnce *sync.Once + totalChunks int + chunkID int + uuid string + chunkRecorder *chunkRecorder + transaction *transaction + reservedMem int64 } // stopBlock can be invoked multiple times safety @@ -1459,55 +1443,6 @@ func (sr *sendRequest) stopBlock() { }) } -//nolint:all -func (sr *sendRequest) done(msgID MessageID, err error) { - if err == nil { - sr.producer.metrics.PublishLatency.Observe(float64(time.Now().UnixNano()-sr.publishTime.UnixNano()) / 1.0e9) - sr.producer.metrics.MessagesPublished.Inc() - sr.producer.metrics.BytesPublished.Add(float64(sr.reservedMem)) - } - - if err != nil { - sr.producer.log.WithError(err). - WithField("size", sr.reservedMem). - WithField("properties", sr.msg.Properties) - } - - if err == errSendTimeout { - sr.producer.metrics.PublishErrorsTimeout.Inc() - } - - if err == errMessageTooLarge { - sr.producer.metrics.PublishErrorsMsgTooLarge.Inc() - } - - if sr.semaphore != nil { - for i := 0; i < sr.reservedSemaphore; i++ { - sr.semaphore.Release() - } - sr.producer.metrics.MessagesPending.Sub(float64(sr.reservedSemaphore)) - } - - if sr.memLimit != nil { - sr.memLimit.ReleaseMemory(sr.reservedMem) - sr.producer.metrics.BytesPending.Sub(float64(sr.reservedMem)) - } - - if sr.totalChunks <= 1 || sr.chunkID == sr.totalChunks-1 { - sr.callbackOnce.Do(func() { - runCallback(sr.callback, msgID, sr.msg, err) - }) - - if sr.transaction != nil { - sr.transaction.endSendOrAckOp(err) - } - - if sr.producer.options.Interceptors != nil && err == nil { - sr.producer.options.Interceptors.OnSendAcknowledgement(sr.producer, sr.msg, msgID) - } - } -} - type closeProducer struct { doneCh chan struct{} } From af56e6031d8021ddcff188f94c4d744eba3b3211 Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 24 Oct 2023 21:27:02 +0800 Subject: [PATCH 285/348] refactor: factor out validateMsg (#1117) Signed-off-by: tison --- pulsar/producer_partition.go | 40 ++++++++++++++++++++++-------------- 1 file changed, 25 insertions(+), 15 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 82123f988a..62c180ca92 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -495,14 +495,6 @@ func (p *partitionProducer) internalSend(request *sendRequest) { return } - if p.options.DisableMultiSchema { - if msg.Schema != nil && p.options.Schema != nil && - msg.Schema.GetSchemaInfo().hash() != p.options.Schema.GetSchemaInfo().hash() { - runCallback(request.callback, nil, request.msg, fmt.Errorf("msg schema can not match with producer schema")) - p.log.WithError(err).Errorf("The producer %s of the topic %s is disabled the `MultiSchema`", p.producerName, p.topic) - return - } - } var schema Schema var schemaVersion []byte if msg.Schema != nil { @@ -1121,17 +1113,35 @@ func (p *partitionProducer) SendAsync(ctx context.Context, msg *ProducerMessage, p.internalSendAsync(ctx, msg, callback, false) } -func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *ProducerMessage, - callback func(MessageID, *ProducerMessage, error), flushImmediately bool) { +func (p *partitionProducer) validateMsg(msg *ProducerMessage) error { if msg == nil { - p.log.Error("Message is nil") - runCallback(callback, nil, msg, newError(InvalidMessage, "Message is nil")) - return + return newError(InvalidMessage, "Message is nil") } if msg.Value != nil && msg.Payload != nil { - p.log.Error("Can not set Value and Payload both") - runCallback(callback, nil, msg, newError(InvalidMessage, "Can not set Value and Payload both")) + return newError(InvalidMessage, "Can not set Value and Payload both") + } + + if p.options.DisableMultiSchema { + if msg.Schema != nil && p.options.Schema != nil && + msg.Schema.GetSchemaInfo().hash() != p.options.Schema.GetSchemaInfo().hash() { + p.log.Errorf("The producer %s of the topic %s is disabled the `MultiSchema`", p.producerName, p.topic) + return fmt.Errorf("msg schema can not match with producer schema") + } + } + + return nil +} + +func (p *partitionProducer) internalSendAsync( + ctx context.Context, + msg *ProducerMessage, + callback func(MessageID, *ProducerMessage, error), + flushImmediately bool, +) { + if err := p.validateMsg(msg); err != nil { + p.log.Error(err) + runCallback(callback, nil, msg, err) return } From 1b135f44fb5747be3cd5aed2ce9d07e67b943df9 Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 24 Oct 2023 23:03:29 +0800 Subject: [PATCH 286/348] refactor: factor out prepareTransaction (#1118) Signed-off-by: tison Co-authored-by: gunli --- pulsar/producer_partition.go | 79 +++++++++++++++++++----------------- 1 file changed, 42 insertions(+), 37 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 62c180ca92..ac07341d3f 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1133,6 +1133,36 @@ func (p *partitionProducer) validateMsg(msg *ProducerMessage) error { return nil } +func (p *partitionProducer) prepareTransaction(sr *sendRequest) error { + if sr.msg.Transaction == nil { + return nil + } + + txn := (sr.msg.Transaction).(*transaction) + if txn.state != TxnOpen { + p.log.WithField("state", txn.state).Error("Failed to send message" + + " by a non-open transaction.") + return newError(InvalidStatus, "Failed to send message by a non-open transaction.") + } + + if err := txn.registerProducerTopic(p.topic); err != nil { + return err + } + + if err := txn.registerSendOrAckOp(); err != nil { + return err + } + + sr.transaction = txn + callback := sr.callback + sr.callback = func(id MessageID, producerMessage *ProducerMessage, err error) { + runCallback(callback, id, producerMessage, err) + txn.endSendOrAckOp(err) + } + + return nil +} + func (p *partitionProducer) internalSendAsync( ctx context.Context, msg *ProducerMessage, @@ -1145,56 +1175,31 @@ func (p *partitionProducer) internalSendAsync( return } - // Register transaction operation to transaction and the transaction coordinator. - var newCallback func(MessageID, *ProducerMessage, error) - var txn *transaction - if msg.Transaction != nil { - transactionImpl := (msg.Transaction).(*transaction) - txn = transactionImpl - if transactionImpl.state != TxnOpen { - p.log.WithField("state", transactionImpl.state).Error("Failed to send message" + - " by a non-open transaction.") - runCallback(callback, nil, msg, newError(InvalidStatus, "Failed to send message by a non-open transaction.")) - return - } - - if err := transactionImpl.registerProducerTopic(p.topic); err != nil { - runCallback(callback, nil, msg, err) - return - } - if err := transactionImpl.registerSendOrAckOp(); err != nil { - runCallback(callback, nil, msg, err) - return - } - newCallback = func(id MessageID, producerMessage *ProducerMessage, err error) { - runCallback(callback, id, producerMessage, err) - transactionImpl.endSendOrAckOp(err) - } - } else { - newCallback = callback - } - if p.getProducerState() != producerReady { - // Producer is closing - runCallback(newCallback, nil, msg, errProducerClosed) - return - } - // bc only works when DisableBlockIfQueueFull is false bc := make(chan struct{}) - // callbackOnce make sure the callback is only invoked once in chunking callbackOnce := &sync.Once{} sr := &sendRequest{ ctx: ctx, msg: msg, - callback: newCallback, + callback: callback, callbackOnce: callbackOnce, flushImmediately: flushImmediately, publishTime: time.Now(), blockCh: bc, closeBlockChOnce: &sync.Once{}, - transaction: txn, } + if err := p.prepareTransaction(sr); err != nil { + runCallback(sr.callback, nil, msg, err) + return + } + + if p.getProducerState() != producerReady { + // Producer is closing + runCallback(sr.callback, nil, msg, errProducerClosed) + return + } + p.options.Interceptors.BeforeSend(p, msg) p.dataChan <- sr From e3046f81f20147cbba83609b7c31be39f8d8c7eb Mon Sep 17 00:00:00 2001 From: Oliver Muir Date: Tue, 24 Oct 2023 19:26:13 +0100 Subject: [PATCH 287/348] chore: Complete comment on ProducerInterceptor interface BeforeSend method (#1119) Signed-off-by: Oliver Muir Co-authored-by: Oliver Muir --- pulsar/producer_interceptor.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar/producer_interceptor.go b/pulsar/producer_interceptor.go index cb2cc152f5..e18994cfcd 100644 --- a/pulsar/producer_interceptor.go +++ b/pulsar/producer_interceptor.go @@ -19,6 +19,7 @@ package pulsar type ProducerInterceptor interface { // BeforeSend This is called before send the message to the brokers. This method is allowed to modify the + // message. BeforeSend(producer Producer, message *ProducerMessage) // OnSendAcknowledgement This method is called when the message sent to the broker has been acknowledged, From 793008e7f3b233bfa64e8c22b7cabe10aa334bdb Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 25 Oct 2023 15:39:23 +0800 Subject: [PATCH 288/348] refactor: prepare sendrequest and move to internalSendAsync (#1120) Signed-off-by: tison Co-authored-by: gunli --- pulsar/message_chunking_test.go | 31 ++- pulsar/producer_partition.go | 457 +++++++++++++++++--------------- 2 files changed, 259 insertions(+), 229 deletions(-) diff --git a/pulsar/message_chunking_test.go b/pulsar/message_chunking_test.go index ee3ab17760..fbdcaa0ceb 100644 --- a/pulsar/message_chunking_test.go +++ b/pulsar/message_chunking_test.go @@ -552,26 +552,41 @@ func sendSingleChunk(p Producer, uuid string, chunkID int, totalChunks int) { msg := &ProducerMessage{ Payload: []byte(fmt.Sprintf("chunk-%s-%d|", uuid, chunkID)), } + wholePayload := msg.Payload producerImpl := p.(*producer).producers[0].(*partitionProducer) - mm := producerImpl.genMetadata(msg, len(msg.Payload), time.Now()) + mm := producerImpl.genMetadata(msg, len(wholePayload), time.Now()) mm.Uuid = proto.String(uuid) mm.NumChunksFromMsg = proto.Int32(int32(totalChunks)) - mm.TotalChunkMsgSize = proto.Int32(int32(len(msg.Payload))) + mm.TotalChunkMsgSize = proto.Int32(int32(len(wholePayload))) mm.ChunkId = proto.Int32(int32(chunkID)) producerImpl.updateMetadataSeqID(mm, msg) - - doneCh := make(chan struct{}) producerImpl.internalSingleSend( mm, msg.Payload, &sendRequest{ callback: func(id MessageID, producerMessage *ProducerMessage, err error) { - close(doneCh) }, - msg: msg, + ctx: context.Background(), + msg: msg, + flushImmediately: true, + totalChunks: totalChunks, + chunkID: chunkID, + uuid: uuid, + chunkRecorder: newChunkRecorder(), + transaction: nil, + reservedMem: 0, + sendAsBatch: false, + schema: nil, + schemaVersion: nil, + uncompressedPayload: wholePayload, + uncompressedSize: int64(len(wholePayload)), + compressedPayload: wholePayload, + compressedSize: len(wholePayload), + payloadChunkSize: internal.MaxMessageSize - proto.Size(mm), + mm: mm, + deliverAt: time.Now(), + maxMessageSize: internal.MaxMessageSize, }, uint32(internal.MaxMessageSize), ) - - <-doneCh } diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index ac07341d3f..b00ed6b53e 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -478,201 +478,31 @@ func runCallback(cb func(MessageID, *ProducerMessage, error), id MessageID, msg cb(id, msg, err) } -func (p *partitionProducer) internalSend(request *sendRequest) { - p.log.Debug("Received send request: ", *request.msg) +func (p *partitionProducer) internalSend(sr *sendRequest) { + p.log.Debug("Received send request: ", *sr.msg) - msg := request.msg - - // read payload from message - uncompressedPayload := msg.Payload + msg := sr.msg - var schemaPayload []byte - var err error - - // The block chan must be closed when returned with exception - defer request.stopBlock() - if !p.canAddToQueue(request) { + if !p.canAddToQueue(sr) { return } - var schema Schema - var schemaVersion []byte - if msg.Schema != nil { - schema = msg.Schema - } else if p.options.Schema != nil { - schema = p.options.Schema - } - if msg.Value != nil { - // payload and schema are mutually exclusive - // try to get payload from schema value only if payload is not set - if uncompressedPayload == nil && schema != nil { - schemaPayload, err = schema.Encode(msg.Value) - if err != nil { - runCallback(request.callback, nil, request.msg, newError(SchemaFailure, err.Error())) - p.log.WithError(err).Errorf("Schema encode message failed %s", msg.Value) - return - } - } - } - if uncompressedPayload == nil { - uncompressedPayload = schemaPayload - } - - if schema != nil { - schemaVersion = p.schemaCache.Get(schema.GetSchemaInfo()) - if schemaVersion == nil { - schemaVersion, err = p.getOrCreateSchema(schema.GetSchemaInfo()) - if err != nil { - p.log.WithError(err).Error("get schema version fail") - runCallback(request.callback, nil, request.msg, fmt.Errorf("get schema version fail, err: %w", err)) - return - } - p.schemaCache.Put(schema.GetSchemaInfo(), schemaVersion) - } - } - - uncompressedSize := len(uncompressedPayload) - // try to reserve memory for uncompressedPayload - if !p.canReserveMem(request, int64(uncompressedSize)) { + if !p.canReserveMem(sr, sr.uncompressedSize) { return } - deliverAt := msg.DeliverAt - if msg.DeliverAfter.Nanoseconds() > 0 { - deliverAt = time.Now().Add(msg.DeliverAfter) - } - - // set default ReplicationClusters when DisableReplication - if msg.DisableReplication { - msg.ReplicationClusters = []string{"__local__"} - } - - mm := p.genMetadata(msg, uncompressedSize, deliverAt) - - sendAsBatch := !p.options.DisableBatching && - msg.ReplicationClusters == nil && - deliverAt.UnixNano() < 0 - - // Once the batching is enabled, it can close blockCh early to make block finish - if sendAsBatch { - request.stopBlock() - } else { - // update sequence id for metadata, make the size of msgMetadata more accurate - // batch sending will update sequence ID in the BatchBuilder - p.updateMetadataSeqID(mm, msg) - } - - maxMessageSize := int(p._getConn().GetMaxMessageSize()) - - // compress payload if not batching - var compressedPayload []byte - var compressedSize int - var checkSize int - if !sendAsBatch { - compressedPayload = p.compressionProvider.Compress(nil, uncompressedPayload) - compressedSize = len(compressedPayload) - checkSize = compressedSize - - // set the compress type in msgMetaData - compressionType := pb.CompressionType(p.options.CompressionType) - if compressionType != pb.CompressionType_NONE { - mm.Compression = &compressionType - } - } else { - // final check for batching message is in serializeMessage - // this is a double check - checkSize = uncompressedSize - } - - // if msg is too large and chunking is disabled - if checkSize > maxMessageSize && !p.options.EnableChunking { - p.releaseSemaphoreAndMem(int64(uncompressedSize)) - runCallback(request.callback, nil, request.msg, errMessageTooLarge) - p.log.WithError(errMessageTooLarge). - WithField("size", checkSize). - WithField("properties", msg.Properties). - Errorf("MaxMessageSize %d", maxMessageSize) + if err := p.updateChunkInfo(sr); err != nil { + p.releaseSemaphoreAndMem(sr.uncompressedSize) + runCallback(sr.callback, nil, sr.msg, err) p.metrics.PublishErrorsMsgTooLarge.Inc() return } - var totalChunks int - // max chunk payload size - var payloadChunkSize int - if sendAsBatch || !p.options.EnableChunking { - totalChunks = 1 - payloadChunkSize = int(p._getConn().GetMaxMessageSize()) - } else { - payloadChunkSize = int(p._getConn().GetMaxMessageSize()) - proto.Size(mm) - if payloadChunkSize <= 0 { - p.releaseSemaphoreAndMem(int64(uncompressedSize)) - runCallback(request.callback, nil, msg, errMetaTooLarge) - p.log.WithError(errMetaTooLarge). - WithField("metadata size", proto.Size(mm)). - WithField("properties", msg.Properties). - Errorf("MaxMessageSize %d", int(p._getConn().GetMaxMessageSize())) - p.metrics.PublishErrorsMsgTooLarge.Inc() - return - } - // set ChunkMaxMessageSize - if p.options.ChunkMaxMessageSize != 0 { - payloadChunkSize = int(math.Min(float64(payloadChunkSize), float64(p.options.ChunkMaxMessageSize))) - } - totalChunks = int(math.Max(1, math.Ceil(float64(compressedSize)/float64(payloadChunkSize)))) - } - - // set total chunks to send request - request.totalChunks = totalChunks - - if !sendAsBatch { - if totalChunks > 1 { - var lhs, rhs int - uuid := fmt.Sprintf("%s-%s", p.producerName, strconv.FormatUint(*mm.SequenceId, 10)) - mm.Uuid = proto.String(uuid) - mm.NumChunksFromMsg = proto.Int32(int32(totalChunks)) - mm.TotalChunkMsgSize = proto.Int32(int32(compressedSize)) - cr := newChunkRecorder() - for chunkID := 0; chunkID < totalChunks; chunkID++ { - lhs = chunkID * payloadChunkSize - if rhs = lhs + payloadChunkSize; rhs > compressedSize { - rhs = compressedSize - } - // update chunk id - mm.ChunkId = proto.Int32(int32(chunkID)) - nsr := &sendRequest{ - ctx: request.ctx, - msg: request.msg, - callback: request.callback, - callbackOnce: request.callbackOnce, - publishTime: request.publishTime, - blockCh: request.blockCh, - closeBlockChOnce: request.closeBlockChOnce, - totalChunks: totalChunks, - chunkID: chunkID, - uuid: uuid, - chunkRecorder: cr, - transaction: request.transaction, - reservedMem: int64(rhs - lhs), - } - // the permit of first chunk has acquired - if chunkID != 0 && !p.canAddToQueue(nsr) { - p.releaseSemaphoreAndMem(int64(uncompressedSize - lhs)) - return - } - p.internalSingleSend(mm, compressedPayload[lhs:rhs], nsr, uint32(maxMessageSize)) - } - // close the blockCh when all the chunks acquired permits - request.stopBlock() - } else { - // close the blockCh when totalChunks is 1 (it has acquired permits) - request.stopBlock() - p.internalSingleSend(mm, compressedPayload, request, uint32(maxMessageSize)) - } - } else { - smm := p.genSingleMessageMetadataInBatch(msg, uncompressedSize) + if sr.sendAsBatch { + smm := p.genSingleMessageMetadataInBatch(msg, int(sr.uncompressedSize)) multiSchemaEnabled := !p.options.DisableMultiSchema - added := addRequestToBatch(smm, p, uncompressedPayload, request, msg, deliverAt, schemaVersion, + added := addRequestToBatch(smm, p, sr.uncompressedPayload, sr, msg, sr.deliverAt, sr.schemaVersion, multiSchemaEnabled) if !added { // The current batch is full. flush it and retry @@ -680,21 +510,71 @@ func (p *partitionProducer) internalSend(request *sendRequest) { p.internalFlushCurrentBatch() // after flushing try again to add the current payload - if ok := addRequestToBatch(smm, p, uncompressedPayload, request, msg, deliverAt, schemaVersion, + if ok := addRequestToBatch(smm, p, sr.uncompressedPayload, sr, msg, sr.deliverAt, sr.schemaVersion, multiSchemaEnabled); !ok { - p.releaseSemaphoreAndMem(int64(uncompressedSize)) - runCallback(request.callback, nil, request.msg, errFailAddToBatch) - p.log.WithField("size", uncompressedSize). + p.releaseSemaphoreAndMem(sr.uncompressedSize) + runCallback(sr.callback, nil, sr.msg, errFailAddToBatch) + p.log.WithField("size", sr.uncompressedSize). WithField("properties", msg.Properties). Error("unable to add message to batch") return } } - if request.flushImmediately { - + if sr.flushImmediately { p.internalFlushCurrentBatch() + } + return + } + + if sr.totalChunks <= 1 { + p.internalSingleSend(sr.mm, sr.compressedPayload, sr, uint32(sr.maxMessageSize)) + return + } + var lhs, rhs int + uuid := fmt.Sprintf("%s-%s", p.producerName, strconv.FormatUint(*sr.mm.SequenceId, 10)) + sr.mm.Uuid = proto.String(uuid) + sr.mm.NumChunksFromMsg = proto.Int32(int32(sr.totalChunks)) + sr.mm.TotalChunkMsgSize = proto.Int32(int32(sr.compressedSize)) + cr := newChunkRecorder() + for chunkID := 0; chunkID < sr.totalChunks; chunkID++ { + lhs = chunkID * sr.payloadChunkSize + if rhs = lhs + sr.payloadChunkSize; rhs > sr.compressedSize { + rhs = sr.compressedSize } + // update chunk id + sr.mm.ChunkId = proto.Int32(int32(chunkID)) + nsr := &sendRequest{ + ctx: sr.ctx, + msg: sr.msg, + callback: sr.callback, + callbackOnce: sr.callbackOnce, + publishTime: sr.publishTime, + flushImmediately: sr.flushImmediately, + totalChunks: sr.totalChunks, + chunkID: chunkID, + uuid: uuid, + chunkRecorder: cr, + transaction: sr.transaction, + reservedMem: int64(rhs - lhs), + sendAsBatch: sr.sendAsBatch, + schema: sr.schema, + schemaVersion: sr.schemaVersion, + uncompressedPayload: sr.uncompressedPayload, + uncompressedSize: sr.uncompressedSize, + compressedPayload: sr.compressedPayload, + compressedSize: sr.compressedSize, + payloadChunkSize: sr.payloadChunkSize, + mm: sr.mm, + deliverAt: sr.deliverAt, + maxMessageSize: sr.maxMessageSize, + } + // the permit of first chunk has acquired + if chunkID != 0 && !p.canAddToQueue(nsr) { + p.releaseSemaphoreAndMem(sr.uncompressedSize - int64(lhs)) + return + } + p.internalSingleSend(sr.mm, sr.compressedPayload[lhs:rhs], nsr, uint32(sr.maxMessageSize)) } } @@ -766,8 +646,10 @@ func (p *partitionProducer) updateSingleMessageMetadataSeqID(smm *pb.SingleMessa } } -func (p *partitionProducer) genSingleMessageMetadataInBatch(msg *ProducerMessage, - uncompressedSize int) (smm *pb.SingleMessageMetadata) { +func (p *partitionProducer) genSingleMessageMetadataInBatch( + msg *ProducerMessage, + uncompressedSize int, +) (smm *pb.SingleMessageMetadata) { smm = &pb.SingleMessageMetadata{ PayloadSize: proto.Int32(int32(uncompressedSize)), } @@ -1163,6 +1045,134 @@ func (p *partitionProducer) prepareTransaction(sr *sendRequest) error { return nil } +func (p *partitionProducer) updateSchema(sr *sendRequest) error { + var schema Schema + var schemaVersion []byte + var err error + + if sr.msg.Schema != nil { + schema = sr.msg.Schema + } else if p.options.Schema != nil { + schema = p.options.Schema + } + + if schema == nil { + return nil + } + + schemaVersion = p.schemaCache.Get(schema.GetSchemaInfo()) + if schemaVersion == nil { + schemaVersion, err = p.getOrCreateSchema(schema.GetSchemaInfo()) + if err != nil { + return fmt.Errorf("get schema version fail, err: %w", err) + } + p.schemaCache.Put(schema.GetSchemaInfo(), schemaVersion) + } + + sr.schema = schema + sr.schemaVersion = schemaVersion + return nil +} + +func (p *partitionProducer) updateUncompressedPayload(sr *sendRequest) error { + // read payload from message + sr.uncompressedPayload = sr.msg.Payload + + if sr.msg.Value != nil { + if sr.schema == nil { + p.log.Errorf("Schema encode message failed %s", sr.msg.Value) + return newError(SchemaFailure, "set schema value without setting schema") + } + + // payload and schema are mutually exclusive + // try to get payload from schema value only if payload is not set + schemaPayload, err := sr.schema.Encode(sr.msg.Value) + if err != nil { + p.log.WithError(err).Errorf("Schema encode message failed %s", sr.msg.Value) + return newError(SchemaFailure, err.Error()) + } + + sr.uncompressedPayload = schemaPayload + } + + sr.uncompressedSize = int64(len(sr.uncompressedPayload)) + return nil +} + +func (p *partitionProducer) updateMetaData(sr *sendRequest) { + deliverAt := sr.msg.DeliverAt + if sr.msg.DeliverAfter.Nanoseconds() > 0 { + deliverAt = time.Now().Add(sr.msg.DeliverAfter) + } + + // set default ReplicationClusters when DisableReplication + if sr.msg.DisableReplication { + sr.msg.ReplicationClusters = []string{"__local__"} + } + + sr.mm = p.genMetadata(sr.msg, int(sr.uncompressedSize), deliverAt) + + sr.sendAsBatch = !p.options.DisableBatching && + sr.msg.ReplicationClusters == nil && + deliverAt.UnixNano() < 0 + + if !sr.sendAsBatch { + // update sequence id for metadata, make the size of msgMetadata more accurate + // batch sending will update sequence ID in the BatchBuilder + p.updateMetadataSeqID(sr.mm, sr.msg) + } + + sr.deliverAt = deliverAt +} + +func (p *partitionProducer) updateChunkInfo(sr *sendRequest) error { + checkSize := sr.uncompressedSize + if !sr.sendAsBatch { + sr.compressedPayload = p.compressionProvider.Compress(nil, sr.uncompressedPayload) + sr.compressedSize = len(sr.compressedPayload) + + // set the compress type in msgMetaData + compressionType := pb.CompressionType(p.options.CompressionType) + if compressionType != pb.CompressionType_NONE { + sr.mm.Compression = &compressionType + } + + checkSize = int64(sr.compressedSize) + } + + sr.maxMessageSize = p._getConn().GetMaxMessageSize() + + // if msg is too large and chunking is disabled + if checkSize > int64(sr.maxMessageSize) && !p.options.EnableChunking { + p.log.WithError(errMessageTooLarge). + WithField("size", checkSize). + WithField("properties", sr.msg.Properties). + Errorf("MaxMessageSize %d", sr.maxMessageSize) + return errMessageTooLarge + } + + if sr.sendAsBatch || !p.options.EnableChunking { + sr.totalChunks = 1 + sr.payloadChunkSize = int(sr.maxMessageSize) + } else { + sr.payloadChunkSize = int(sr.maxMessageSize) - proto.Size(sr.mm) + if sr.payloadChunkSize <= 0 { + p.log.WithError(errMetaTooLarge). + WithField("metadata size", proto.Size(sr.mm)). + WithField("properties", sr.msg.Properties). + Errorf("MaxMessageSize %d", int(p._getConn().GetMaxMessageSize())) + return errMetaTooLarge + } + // set ChunkMaxMessageSize + if p.options.ChunkMaxMessageSize != 0 { + sr.payloadChunkSize = int(math.Min(float64(sr.payloadChunkSize), float64(p.options.ChunkMaxMessageSize))) + } + sr.totalChunks = int(math.Max(1, math.Ceil(float64(sr.compressedSize)/float64(sr.payloadChunkSize)))) + } + + return nil +} + func (p *partitionProducer) internalSendAsync( ctx context.Context, msg *ProducerMessage, @@ -1175,19 +1185,13 @@ func (p *partitionProducer) internalSendAsync( return } - // bc only works when DisableBlockIfQueueFull is false - bc := make(chan struct{}) - // callbackOnce make sure the callback is only invoked once in chunking - callbackOnce := &sync.Once{} sr := &sendRequest{ ctx: ctx, msg: msg, callback: callback, - callbackOnce: callbackOnce, + callbackOnce: &sync.Once{}, flushImmediately: flushImmediately, publishTime: time.Now(), - blockCh: bc, - closeBlockChOnce: &sync.Once{}, } if err := p.prepareTransaction(sr); err != nil { runCallback(sr.callback, nil, msg, err) @@ -1202,12 +1206,21 @@ func (p *partitionProducer) internalSendAsync( p.options.Interceptors.BeforeSend(p, msg) - p.dataChan <- sr + if err := p.updateSchema(sr); err != nil { + p.log.Error(err) + runCallback(sr.callback, nil, msg, err) + return + } - if !p.options.DisableBlockIfQueueFull { - // block if queue full - <-bc + if err := p.updateUncompressedPayload(sr); err != nil { + p.log.Error(err) + runCallback(sr.callback, nil, msg, err) + return } + + p.updateMetaData(sr) + + p.dataChan <- sr } func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) { @@ -1435,27 +1448,29 @@ func (p *partitionProducer) Close() { } type sendRequest struct { - ctx context.Context - msg *ProducerMessage - callback func(MessageID, *ProducerMessage, error) - callbackOnce *sync.Once - publishTime time.Time - flushImmediately bool - blockCh chan struct{} - closeBlockChOnce *sync.Once - totalChunks int - chunkID int - uuid string - chunkRecorder *chunkRecorder - transaction *transaction - reservedMem int64 -} - -// stopBlock can be invoked multiple times safety -func (sr *sendRequest) stopBlock() { - sr.closeBlockChOnce.Do(func() { - close(sr.blockCh) - }) + ctx context.Context + msg *ProducerMessage + callback func(MessageID, *ProducerMessage, error) + callbackOnce *sync.Once + publishTime time.Time + flushImmediately bool + totalChunks int + chunkID int + uuid string + chunkRecorder *chunkRecorder + transaction *transaction + reservedMem int64 + sendAsBatch bool + schema Schema + schemaVersion []byte + uncompressedPayload []byte + uncompressedSize int64 + compressedPayload []byte + compressedSize int + payloadChunkSize int + mm *pb.MessageMetadata + deliverAt time.Time + maxMessageSize int32 } type closeProducer struct { From 7c14fb7a8e7c783db9d5738c3b2aa65d56f2db82 Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 25 Oct 2023 19:44:33 +0800 Subject: [PATCH 289/348] fix: normalize all send request resource release into sr.done (#1121) Signed-off-by: tison Co-authored-by: gunli --- pulsar/message_chunking_test.go | 15 +- pulsar/producer_partition.go | 419 +++++++++++++++++--------------- pulsar/producer_test.go | 6 +- pulsar/transaction_test.go | 1 - 4 files changed, 228 insertions(+), 213 deletions(-) diff --git a/pulsar/message_chunking_test.go b/pulsar/message_chunking_test.go index fbdcaa0ceb..59fdb5ec61 100644 --- a/pulsar/message_chunking_test.go +++ b/pulsar/message_chunking_test.go @@ -24,6 +24,7 @@ import ( "math/rand" "net/http" "strings" + "sync" "testing" "time" @@ -531,12 +532,13 @@ func TestChunkBlockIfQueueFull(t *testing.T) { assert.NotNil(t, producer) defer producer.Close() + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() // Large messages will be split into 11 chunks, exceeding the length of pending queue - ID, err := producer.Send(context.Background(), &ProducerMessage{ + _, err = producer.Send(ctx, &ProducerMessage{ Payload: createTestMessagePayload(100), }) - assert.NoError(t, err) - assert.NotNil(t, ID) + assert.Error(t, err) } func createTestMessagePayload(size int) []byte { @@ -566,18 +568,15 @@ func sendSingleChunk(p Producer, uuid string, chunkID int, totalChunks int) { &sendRequest{ callback: func(id MessageID, producerMessage *ProducerMessage, err error) { }, + callbackOnce: &sync.Once{}, ctx: context.Background(), msg: msg, + producer: producerImpl, flushImmediately: true, totalChunks: totalChunks, chunkID: chunkID, uuid: uuid, chunkRecorder: newChunkRecorder(), - transaction: nil, - reservedMem: 0, - sendAsBatch: false, - schema: nil, - schemaVersion: nil, uncompressedPayload: wholePayload, uncompressedSize: int64(len(wholePayload)), compressedPayload: wholePayload, diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index b00ed6b53e..f606fe0575 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -251,7 +251,7 @@ func (p *partitionProducer) grabCnx() error { res, err := p.client.rpcClient.Request(lr.LogicalAddr, lr.PhysicalAddr, id, pb.BaseCommand_PRODUCER, cmdProducer) if err != nil { p.log.WithError(err).Error("Failed to create producer at send PRODUCER request") - if err == internal.ErrRequestTimeOut { + if errors.Is(err, internal.ErrRequestTimeOut) { id := p.client.rpcClient.NewRequestID() _, _ = p.client.rpcClient.Request(lr.LogicalAddr, lr.PhysicalAddr, id, pb.BaseCommand_CLOSE_PRODUCER, &pb.CommandCloseProducer{ @@ -481,45 +481,28 @@ func runCallback(cb func(MessageID, *ProducerMessage, error), id MessageID, msg func (p *partitionProducer) internalSend(sr *sendRequest) { p.log.Debug("Received send request: ", *sr.msg) - msg := sr.msg - - if !p.canAddToQueue(sr) { - return - } - - // try to reserve memory for uncompressedPayload - if !p.canReserveMem(sr, sr.uncompressedSize) { - return - } - - if err := p.updateChunkInfo(sr); err != nil { - p.releaseSemaphoreAndMem(sr.uncompressedSize) - runCallback(sr.callback, nil, sr.msg, err) - p.metrics.PublishErrorsMsgTooLarge.Inc() - return - } - if sr.sendAsBatch { - smm := p.genSingleMessageMetadataInBatch(msg, int(sr.uncompressedSize)) + smm := p.genSingleMessageMetadataInBatch(sr.msg, int(sr.uncompressedSize)) multiSchemaEnabled := !p.options.DisableMultiSchema - added := addRequestToBatch(smm, p, sr.uncompressedPayload, sr, msg, sr.deliverAt, sr.schemaVersion, - multiSchemaEnabled) + + added := addRequestToBatch( + smm, p, sr.uncompressedPayload, sr, sr.msg, sr.deliverAt, sr.schemaVersion, multiSchemaEnabled) if !added { // The current batch is full. flush it and retry - p.internalFlushCurrentBatch() // after flushing try again to add the current payload - if ok := addRequestToBatch(smm, p, sr.uncompressedPayload, sr, msg, sr.deliverAt, sr.schemaVersion, - multiSchemaEnabled); !ok { - p.releaseSemaphoreAndMem(sr.uncompressedSize) - runCallback(sr.callback, nil, sr.msg, errFailAddToBatch) + ok := addRequestToBatch( + smm, p, sr.uncompressedPayload, sr, sr.msg, sr.deliverAt, sr.schemaVersion, multiSchemaEnabled) + if !ok { p.log.WithField("size", sr.uncompressedSize). - WithField("properties", msg.Properties). + WithField("properties", sr.msg.Properties). Error("unable to add message to batch") + sr.done(nil, errFailAddToBatch) return } } + if sr.flushImmediately { p.internalFlushCurrentBatch() } @@ -547,6 +530,7 @@ func (p *partitionProducer) internalSend(sr *sendRequest) { nsr := &sendRequest{ ctx: sr.ctx, msg: sr.msg, + producer: sr.producer, callback: sr.callback, callbackOnce: sr.callbackOnce, publishTime: sr.publishTime, @@ -556,6 +540,8 @@ func (p *partitionProducer) internalSend(sr *sendRequest) { uuid: uuid, chunkRecorder: cr, transaction: sr.transaction, + memLimit: sr.memLimit, + semaphore: sr.semaphore, reservedMem: int64(rhs - lhs), sendAsBatch: sr.sendAsBatch, schema: sr.schema, @@ -569,12 +555,8 @@ func (p *partitionProducer) internalSend(sr *sendRequest) { deliverAt: sr.deliverAt, maxMessageSize: sr.maxMessageSize, } - // the permit of first chunk has acquired - if chunkID != 0 && !p.canAddToQueue(nsr) { - p.releaseSemaphoreAndMem(sr.uncompressedSize - int64(lhs)) - return - } - p.internalSingleSend(sr.mm, sr.compressedPayload[lhs:rhs], nsr, uint32(sr.maxMessageSize)) + + p.internalSingleSend(nsr.mm, nsr.compressedPayload[lhs:rhs], nsr, uint32(nsr.maxMessageSize)) } } @@ -675,11 +657,13 @@ func (p *partitionProducer) genSingleMessageMetadataInBatch( return } -func (p *partitionProducer) internalSingleSend(mm *pb.MessageMetadata, +func (p *partitionProducer) internalSingleSend( + mm *pb.MessageMetadata, compressedPayload []byte, - request *sendRequest, - maxMessageSize uint32) { - msg := request.msg + sr *sendRequest, + maxMessageSize uint32, +) { + msg := sr.msg payloadBuf := internal.NewBuffer(len(compressedPayload)) payloadBuf.Write(compressedPayload) @@ -694,8 +678,8 @@ func (p *partitionProducer) internalSingleSend(mm *pb.MessageMetadata, var mostSigBits uint64 var leastSigBits uint64 - if request.transaction != nil { - txnID := request.transaction.GetTxnID() + if sr.transaction != nil { + txnID := sr.transaction.GetTxnID() useTxn = true mostSigBits = txnID.MostSigBits leastSigBits = txnID.LeastSigBits @@ -715,8 +699,7 @@ func (p *partitionProducer) internalSingleSend(mm *pb.MessageMetadata, ) if err != nil { - runCallback(request.callback, nil, request.msg, err) - p.releaseSemaphoreAndMem(request.reservedMem) + sr.done(nil, err) p.log.WithError(err).Errorf("Single message serialize failed %s", msg.Value) return } @@ -725,7 +708,7 @@ func (p *partitionProducer) internalSingleSend(mm *pb.MessageMetadata, sentAt: time.Now(), buffer: buffer, sequenceID: sid, - sendRequests: []interface{}{request}, + sendRequests: []interface{}{sr}, }) p._getConn().WriteData(buffer) } @@ -756,15 +739,14 @@ func (p *partitionProducer) internalFlushCurrentBatch() { if err != nil { for _, cb := range callbacks { if sr, ok := cb.(*sendRequest); ok { - runCallback(sr.callback, nil, sr.msg, err) + sr.done(nil, err) } } + if errors.Is(err, internal.ErrExceedMaxMessageSize) { - p.log.WithError(errMessageTooLarge). - Errorf("internal err: %s", err) - p.metrics.PublishErrorsMsgTooLarge.Inc() - return + p.log.WithError(errMessageTooLarge).Errorf("internal err: %s", err) } + return } @@ -853,25 +835,7 @@ func (p *partitionProducer) failTimeoutMessages() { for _, i := range pi.sendRequests { sr := i.(*sendRequest) - if sr.msg != nil { - size := len(sr.msg.Payload) - p.releaseSemaphoreAndMem(sr.reservedMem) - p.metrics.MessagesPending.Dec() - p.metrics.BytesPending.Sub(float64(size)) - p.metrics.PublishErrorsTimeout.Inc() - p.log.WithError(errSendTimeout). - WithField("size", size). - WithField("properties", sr.msg.Properties) - } - - if sr.callback != nil { - sr.callbackOnce.Do(func() { - runCallback(sr.callback, nil, sr.msg, errSendTimeout) - }) - } - if sr.transaction != nil { - sr.transaction.endSendOrAckOp(nil) - } + sr.done(nil, errSendTimeout) } // flag the sending has completed with error, flush make no effect @@ -899,15 +863,15 @@ func (p *partitionProducer) internalFlushCurrentBatches() { if errs[i] != nil { for _, cb := range callbacks[i] { if sr, ok := cb.(*sendRequest); ok { - runCallback(sr.callback, nil, sr.msg, errs[i]) + sr.done(nil, errs[i]) } } + if errors.Is(errs[i], internal.ErrExceedMaxMessageSize) { - p.log.WithError(errMessageTooLarge). - Errorf("internal err: %s", errs[i]) - p.metrics.PublishErrorsMsgTooLarge.Inc() + p.log.WithError(errMessageTooLarge).Errorf("internal err: %s", errs[i]) return } + continue } if batchesData[i] == nil { @@ -1036,12 +1000,6 @@ func (p *partitionProducer) prepareTransaction(sr *sendRequest) error { } sr.transaction = txn - callback := sr.callback - sr.callback = func(id MessageID, producerMessage *ProducerMessage, err error) { - runCallback(callback, id, producerMessage, err) - txn.endSendOrAckOp(err) - } - return nil } @@ -1188,19 +1146,21 @@ func (p *partitionProducer) internalSendAsync( sr := &sendRequest{ ctx: ctx, msg: msg, + producer: p, callback: callback, callbackOnce: &sync.Once{}, flushImmediately: flushImmediately, publishTime: time.Now(), + chunkID: -1, } + if err := p.prepareTransaction(sr); err != nil { - runCallback(sr.callback, nil, msg, err) + sr.done(nil, err) return } if p.getProducerState() != producerReady { - // Producer is closing - runCallback(sr.callback, nil, msg, errProducerClosed) + sr.done(nil, errProducerClosed) return } @@ -1208,18 +1168,30 @@ func (p *partitionProducer) internalSendAsync( if err := p.updateSchema(sr); err != nil { p.log.Error(err) - runCallback(sr.callback, nil, msg, err) + sr.done(nil, err) return } if err := p.updateUncompressedPayload(sr); err != nil { p.log.Error(err) - runCallback(sr.callback, nil, msg, err) + sr.done(nil, err) return } p.updateMetaData(sr) + if err := p.updateChunkInfo(sr); err != nil { + p.log.Error(err) + sr.done(nil, err) + return + } + + if err := p.reserveResources(sr); err != nil { + p.log.Error(err) + sr.done(nil, err) + return + } + p.dataChan <- sr } @@ -1257,55 +1229,40 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) for idx, i := range pi.sendRequests { sr := i.(*sendRequest) atomic.StoreInt64(&p.lastSequenceID, int64(pi.sequenceID)) - p.releaseSemaphoreAndMem(sr.reservedMem) - p.metrics.PublishLatency.Observe(float64(now-sr.publishTime.UnixNano()) / 1.0e9) - p.metrics.MessagesPublished.Inc() - p.metrics.MessagesPending.Dec() - payloadSize := float64(len(sr.msg.Payload)) - p.metrics.BytesPublished.Add(payloadSize) - p.metrics.BytesPending.Sub(payloadSize) - - if sr.callback != nil || len(p.options.Interceptors) > 0 { - msgID := newMessageID( - int64(response.MessageId.GetLedgerId()), - int64(response.MessageId.GetEntryId()), - int32(idx), - p.partitionIdx, - batchSize, - ) - - if sr.totalChunks > 1 { - if sr.chunkID == 0 { - sr.chunkRecorder.setFirstChunkID( - &messageID{ - int64(response.MessageId.GetLedgerId()), - int64(response.MessageId.GetEntryId()), - -1, - p.partitionIdx, - 0, - }) - } else if sr.chunkID == sr.totalChunks-1 { - sr.chunkRecorder.setLastChunkID( - &messageID{ - int64(response.MessageId.GetLedgerId()), - int64(response.MessageId.GetEntryId()), - -1, - p.partitionIdx, - 0, - }) - // use chunkMsgID to set msgID - msgID = &sr.chunkRecorder.chunkedMsgID - } - } - if sr.totalChunks <= 1 || sr.chunkID == sr.totalChunks-1 { - runCallback(sr.callback, msgID, sr.msg, nil) - p.options.Interceptors.OnSendAcknowledgement(p, sr.msg, msgID) + msgID := newMessageID( + int64(response.MessageId.GetLedgerId()), + int64(response.MessageId.GetEntryId()), + int32(idx), + p.partitionIdx, + batchSize, + ) + + if sr.totalChunks > 1 { + if sr.chunkID == 0 { + sr.chunkRecorder.setFirstChunkID( + &messageID{ + int64(response.MessageId.GetLedgerId()), + int64(response.MessageId.GetEntryId()), + -1, + p.partitionIdx, + 0, + }) + } else if sr.chunkID == sr.totalChunks-1 { + sr.chunkRecorder.setLastChunkID( + &messageID{ + int64(response.MessageId.GetLedgerId()), + int64(response.MessageId.GetEntryId()), + -1, + p.partitionIdx, + 0, + }) + // use chunkMsgID to set msgID + msgID = &sr.chunkRecorder.chunkedMsgID } } - if sr.transaction != nil { - sr.transaction.endSendOrAckOp(nil) - } + + sr.done(msgID, nil) } // Mark this pending item as done @@ -1372,24 +1329,7 @@ func (p *partitionProducer) failPendingMessages() { for _, i := range pi.sendRequests { sr := i.(*sendRequest) - if sr.msg != nil { - size := len(sr.msg.Payload) - p.releaseSemaphoreAndMem(sr.reservedMem) - p.metrics.MessagesPending.Dec() - p.metrics.BytesPending.Sub(float64(size)) - p.log.WithError(errProducerClosed). - WithField("size", size). - WithField("properties", sr.msg.Properties) - } - - if sr.callback != nil { - sr.callbackOnce.Do(func() { - runCallback(sr.callback, nil, sr.msg, errProducerClosed) - }) - } - if sr.transaction != nil { - sr.transaction.endSendOrAckOp(nil) - } + sr.done(nil, errProducerClosed) } // flag the sending has completed with error, flush make no effect @@ -1448,19 +1388,29 @@ func (p *partitionProducer) Close() { } type sendRequest struct { - ctx context.Context - msg *ProducerMessage - callback func(MessageID, *ProducerMessage, error) - callbackOnce *sync.Once - publishTime time.Time - flushImmediately bool - totalChunks int - chunkID int - uuid string - chunkRecorder *chunkRecorder - transaction *transaction - reservedMem int64 + ctx context.Context + msg *ProducerMessage + producer *partitionProducer + callback func(MessageID, *ProducerMessage, error) + callbackOnce *sync.Once + publishTime time.Time + flushImmediately bool + totalChunks int + chunkID int + uuid string + chunkRecorder *chunkRecorder + + /// resource management + + memLimit internal.MemoryLimitController + reservedMem int64 + semaphore internal.Semaphore + reservedSemaphore int + + /// convey settable state + sendAsBatch bool + transaction *transaction schema Schema schemaVersion []byte uncompressedPayload []byte @@ -1473,6 +1423,116 @@ type sendRequest struct { maxMessageSize int32 } +func (sr *sendRequest) done(msgID MessageID, err error) { + if err == nil { + sr.producer.metrics.PublishLatency.Observe(float64(time.Now().UnixNano()-sr.publishTime.UnixNano()) / 1.0e9) + sr.producer.metrics.MessagesPublished.Inc() + sr.producer.metrics.BytesPublished.Add(float64(sr.reservedMem)) + + if sr.totalChunks <= 1 || sr.chunkID == sr.totalChunks-1 { + if sr.producer.options.Interceptors != nil { + sr.producer.options.Interceptors.OnSendAcknowledgement(sr.producer, sr.msg, msgID) + } + } + } + + if err != nil { + sr.producer.log.WithError(err). + WithField("size", sr.reservedMem). + WithField("properties", sr.msg.Properties) + } + + if errors.Is(err, errSendTimeout) { + sr.producer.metrics.PublishErrorsTimeout.Inc() + } + + if errors.Is(err, errMessageTooLarge) { + sr.producer.metrics.PublishErrorsMsgTooLarge.Inc() + } + + if sr.semaphore != nil { + sr.semaphore.Release() + sr.producer.metrics.MessagesPending.Dec() + } + + if sr.memLimit != nil { + sr.memLimit.ReleaseMemory(sr.reservedMem) + sr.producer.metrics.BytesPending.Sub(float64(sr.reservedMem)) + } + + // sr.chunkID == -1 means a chunked message is not yet prepared, so that we should fail it immediately + if sr.totalChunks <= 1 || sr.chunkID == -1 || sr.chunkID == sr.totalChunks-1 { + sr.callbackOnce.Do(func() { + runCallback(sr.callback, msgID, sr.msg, err) + }) + + if sr.transaction != nil { + sr.transaction.endSendOrAckOp(err) + } + } +} + +func (p *partitionProducer) reserveSemaphore(sr *sendRequest) error { + for i := 0; i < sr.totalChunks; i++ { + if p.options.DisableBlockIfQueueFull { + if !p.publishSemaphore.TryAcquire() { + return errSendQueueIsFull + } + + // update sr.semaphore and sr.reservedSemaphore here so that we can release semaphore in the case + // of that only a part of the chunks acquire succeed + sr.semaphore = p.publishSemaphore + sr.reservedSemaphore++ + p.metrics.MessagesPending.Inc() + } else { + if !p.publishSemaphore.Acquire(sr.ctx) { + return errContextExpired + } + + // update sr.semaphore and sr.reservedSemaphore here so that we can release semaphore in the case + // of that only a part of the chunks acquire succeed + sr.semaphore = p.publishSemaphore + sr.reservedSemaphore++ + p.metrics.MessagesPending.Inc() + } + } + + return nil +} + +func (p *partitionProducer) reserveMem(sr *sendRequest) error { + requiredMem := sr.uncompressedSize + if !sr.sendAsBatch { + requiredMem = int64(sr.compressedSize) + } + + if p.options.DisableBlockIfQueueFull { + if !p.client.memLimit.TryReserveMemory(requiredMem) { + return errMemoryBufferIsFull + } + + } else { + if !p.client.memLimit.ReserveMemory(sr.ctx, requiredMem) { + return errContextExpired + } + } + + sr.memLimit = p.client.memLimit + sr.reservedMem += requiredMem + p.metrics.BytesPending.Add(float64(requiredMem)) + return nil +} + +func (p *partitionProducer) reserveResources(sr *sendRequest) error { + if err := p.reserveSemaphore(sr); err != nil { + return err + } + if err := p.reserveMem(sr); err != nil { + return err + } + return nil +} + type closeProducer struct { doneCh chan struct{} } @@ -1502,53 +1562,12 @@ func (p *partitionProducer) _setConn(conn internal.Connection) { // _getConn returns internal connection field of this partition producer atomically. // Note: should only be called by this partition producer before attempting to use the connection func (p *partitionProducer) _getConn() internal.Connection { - // Invariant: The conn must be non-nil for the lifetime of the partitionProducer. + // Invariant: p.conn must be non-nil for the lifetime of the partitionProducer. // For this reason we leave this cast unchecked and panic() if the // invariant is broken return p.conn.Load().(internal.Connection) } -func (p *partitionProducer) releaseSemaphoreAndMem(size int64) { - p.publishSemaphore.Release() - p.client.memLimit.ReleaseMemory(size) -} - -func (p *partitionProducer) canAddToQueue(sr *sendRequest) bool { - if p.options.DisableBlockIfQueueFull { - if !p.publishSemaphore.TryAcquire() { - runCallback(sr.callback, nil, sr.msg, errSendQueueIsFull) - return false - } - } else { - if !p.publishSemaphore.Acquire(sr.ctx) { - runCallback(sr.callback, nil, sr.msg, errContextExpired) - return false - } - } - p.metrics.MessagesPending.Inc() - return true -} - -func (p *partitionProducer) canReserveMem(sr *sendRequest, size int64) bool { - if p.options.DisableBlockIfQueueFull { - if !p.client.memLimit.TryReserveMemory(size) { - p.publishSemaphore.Release() - runCallback(sr.callback, nil, sr.msg, errMemoryBufferIsFull) - return false - } - - } else { - if !p.client.memLimit.ReserveMemory(sr.ctx, size) { - p.publishSemaphore.Release() - runCallback(sr.callback, nil, sr.msg, errContextExpired) - return false - } - } - sr.reservedMem += size - p.metrics.BytesPending.Add(float64(size)) - return true -} - type chunkRecorder struct { chunkedMsgID chunkMessageID } diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 29ffa7805a..49e225f3e1 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -2079,7 +2079,6 @@ func TestMemLimitRejectProducerMessagesWithSchema(t *testing.T) { } func TestMemLimitRejectProducerMessagesWithChunking(t *testing.T) { - c, err := NewClient(ClientOptions{ URL: serviceURL, MemoryLimitBytes: 5 * 1024, @@ -2136,12 +2135,11 @@ func TestMemLimitRejectProducerMessagesWithChunking(t *testing.T) { SendTimeout: 2 * time.Second, }) - // producer2 will reserve 2*1024 bytes and then release 1024 byte (release the second chunk) - // because it reaches MaxPendingMessages in chunking + // producer3 cannot reserve 2*1024 bytes because it reaches MaxPendingMessages in chunking _, _ = producer3.Send(context.Background(), &ProducerMessage{ Payload: make([]byte, 2*1024), }) - assert.Equal(t, int64(1024), c.(*client).memLimit.CurrentUsage()) + assert.Zero(t, c.(*client).memLimit.CurrentUsage()) } func TestMemLimitContextCancel(t *testing.T) { diff --git a/pulsar/transaction_test.go b/pulsar/transaction_test.go index 385b197e00..74e8dd0c99 100644 --- a/pulsar/transaction_test.go +++ b/pulsar/transaction_test.go @@ -458,7 +458,6 @@ func TestAckChunkMessage(t *testing.T) { // Create transaction and register the send operation. txn, err := client.NewTransaction(time.Hour) require.Nil(t, err) - txn.(*transaction).registerSendOrAckOp() // Create a producer with chunking enabled to send a large message that will be split into chunks. producer, err := client.CreateProducer(ProducerOptions{ From 2a8d15123b817eed5b6d6f141ed504ff3246a02c Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Thu, 26 Oct 2023 19:08:42 +0800 Subject: [PATCH 290/348] [imporve] add func blockIfQueueFull() to encapsulate DisableBlockIfQueueFull (#1122) Co-authored-by: gunli --- pulsar/producer_partition.go | 26 +++++++++++++++----------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index f606fe0575..a435fa7247 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1472,11 +1472,16 @@ func (sr *sendRequest) done(msgID MessageID, err error) { } } +func (p *partitionProducer) blockIfQueueFull() bool { + //DisableBlockIfQueueFull == false means enable block + return !p.options.DisableBlockIfQueueFull +} + func (p *partitionProducer) reserveSemaphore(sr *sendRequest) error { for i := 0; i < sr.totalChunks; i++ { - if p.options.DisableBlockIfQueueFull { - if !p.publishSemaphore.TryAcquire() { - return errSendQueueIsFull + if p.blockIfQueueFull() { + if !p.publishSemaphore.Acquire(sr.ctx) { + return errContextExpired } // update sr.semaphore and sr.reservedSemaphore here so that we can release semaphore in the case @@ -1485,8 +1490,8 @@ func (p *partitionProducer) reserveSemaphore(sr *sendRequest) error { sr.reservedSemaphore++ p.metrics.MessagesPending.Inc() } else { - if !p.publishSemaphore.Acquire(sr.ctx) { - return errContextExpired + if !p.publishSemaphore.TryAcquire() { + return errSendQueueIsFull } // update sr.semaphore and sr.reservedSemaphore here so that we can release semaphore in the case @@ -1506,15 +1511,14 @@ func (p *partitionProducer) reserveMem(sr *sendRequest) error { requiredMem = int64(sr.compressedSize) } - if p.options.DisableBlockIfQueueFull { - if !p.client.memLimit.TryReserveMemory(requiredMem) { - return errMemoryBufferIsFull - } - - } else { + if p.blockIfQueueFull() { if !p.client.memLimit.ReserveMemory(sr.ctx, requiredMem) { return errContextExpired } + } else { + if !p.client.memLimit.TryReserveMemory(requiredMem) { + return errMemoryBufferIsFull + } } sr.memLimit = p.client.memLimit From 620ecf4addfb48de123db3875698669be60d5279 Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Mon, 30 Oct 2023 17:45:48 +0800 Subject: [PATCH 291/348] [improve] make debug log more clearer in ReceivedSendReceipt() (#1123) Co-authored-by: gunli --- pulsar/producer_partition.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index a435fa7247..3b17190d7a 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1206,14 +1206,14 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) if pi.sequenceID < response.GetSequenceId() { // Force connection closing so that messages can be re-transmitted in a new connection - p.log.Warnf("Received ack for %v on sequenceId %v - expected: %v, closing connection", response.GetMessageId(), - response.GetSequenceId(), pi.sequenceID) + p.log.Warnf("Received ack for %v on sequenceId %v - expected: %v, local < remote, closing connection", + response.GetMessageId(), response.GetSequenceId(), pi.sequenceID) p._getConn().Close() return } else if pi.sequenceID > response.GetSequenceId() { // Ignoring the ack since it's referring to a message that has already timed out. - p.log.Warnf("Received ack for %v on sequenceId %v - expected: %v, closing connection", response.GetMessageId(), - response.GetSequenceId(), pi.sequenceID) + p.log.Warnf("Received ack for %v on sequenceId %v - expected: %v, local > remote, ignore it", + response.GetMessageId(), response.GetSequenceId(), pi.sequenceID) return } else { // The ack was indeed for the expected item in the queue, we can remove it and trigger the callback From 0845e7389affa855ee582489be30c6d4ac5766c0 Mon Sep 17 00:00:00 2001 From: ming Date: Wed, 1 Nov 2023 11:35:45 -0400 Subject: [PATCH 292/348] [fix][issue 1098] check batchBuilder in case batch is disabled (#1099) --- pulsar/producer_partition.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 3b17190d7a..e214979dab 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -724,6 +724,13 @@ type pendingItem struct { } func (p *partitionProducer) internalFlushCurrentBatch() { + if p.batchBuilder == nil { + // batch is not enabled + // the batch flush ticker should be stopped but it might still called once + // depends on when stop() is called concurrently + // so we add check to prevent the flow continues on a nil batchBuilder + return + } if p.batchBuilder.IsMultiBatches() { p.internalFlushCurrentBatches() return From 64c87a0a6895ceaeb14ac39432adafbef54d007e Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Sat, 4 Nov 2023 11:14:07 +0800 Subject: [PATCH 293/348] [Fix][Producer] Reconnection backoff logic for producer (#1125) Co-authored-by: gunli --- pulsar/producer_partition.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index e214979dab..8608df3d98 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -396,6 +396,11 @@ func (p *partitionProducer) reconnectToBroker() { maxRetry = int(*p.options.MaxReconnectToBroker) } + var ( + delayReconnectTime time.Duration + defaultBackoff = internal.DefaultBackoff{} + ) + for maxRetry != 0 { if p.getProducerState() != producerReady { // Producer is already closing @@ -403,11 +408,6 @@ func (p *partitionProducer) reconnectToBroker() { return } - var ( - delayReconnectTime time.Duration - defaultBackoff = internal.DefaultBackoff{} - ) - if p.options.BackoffPolicy == nil { delayReconnectTime = defaultBackoff.Next() } else { From 1605a77aff99ccb58332dbc688a09e52d97990d6 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Tue, 7 Nov 2023 18:52:37 +0800 Subject: [PATCH 294/348] Add 0.11.1 change log (#1092) --- CHANGELOG.md | 19 +++++++++++++++++++ VERSION | 2 +- stable.txt | 2 +- 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f1d4839dc2..d7afe2c3a0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,6 +23,25 @@ All notable changes to this project will be documented in this file. +[0.11.1] 2023-09-11 + +- Close consumer resources if the creation fails by @michaeljmarshall in [#1070](/~https://github.com/apache/pulsar-client-go/pull/1070) +- Fix the transaction acknowledgement and send logic for chunked message by @liangyepianzhou in [#1069](/~https://github.com/apache/pulsar-client-go/pull/1069) +- Correct the `SendAsync()` description by @Gleiphir2769 in [#1066](/~https://github.com/apache/pulsar-client-go/pull/1066) +- Fix the panic when try to flush in `DisableBatching=true` by @Gleiphir2769 in [#1065](/~https://github.com/apache/pulsar-client-go/pull/1065) +- Fix client reconnected every authenticationRefreshCheckSeconds when using tls authentication by @jffp113 in [#1062](/~https://github.com/apache/pulsar-client-go/pull/1062) +- Send Close Command on Producer/Consumer create timeout by @michaeljmarshall in [#1061](/~https://github.com/apache/pulsar-client-go/pull/1061) +- Fail all messages that are pending requests when closing by @graysonzeng in [#1059](/~https://github.com/apache/pulsar-client-go/pull/1059) +- Fix the producer flush opertion is not guarantee to flush all messages by @Gleiphir2769 in [#1058](/~https://github.com/apache/pulsar-client-go/pull/1058) +- Fix inaccurate producer mem limit in chunking and schema by @Gleiphir2769 in [#1055](/~https://github.com/apache/pulsar-client-go/pull/1055) +- Fix ctx in `partitionProducer.Send()` is not performing as expected by @Gleiphir2769 in [#1053](/~https://github.com/apache/pulsar-client-go/pull/1053) +- Stop block request even if Value and Payload are both set by @gunli in [#1052](/~https://github.com/apache/pulsar-client-go/pull/1052) +- Simplify the flush logic by @gunli in [#1049](/~https://github.com/apache/pulsar-client-go/pull/1049) +- Check if message is nil by @gunli in [#1047](/~https://github.com/apache/pulsar-client-go/pull/1047) +- Return when registerSendOrAckOp() failed by @gunli in [#1045](/~https://github.com/apache/pulsar-client-go/pull/1045) + +**Full Changelog**: /~https://github.com/apache/pulsar-client-go/compare/v0.11.0...v0.11.1-candidate-1 + [0.11.0] 2023-07-04 ## Features diff --git a/VERSION b/VERSION index 89d00e6236..dbf0637bab 100644 --- a/VERSION +++ b/VERSION @@ -1,3 +1,3 @@ // This version number refers to the currently released version number // Please fix the version when release. -v0.11.0 +v0.11.1 diff --git a/stable.txt b/stable.txt index 66c47738bd..7e38472ca7 100644 --- a/stable.txt +++ b/stable.txt @@ -1,3 +1,3 @@ // This version number refers to the current stable version, generally is `VERSION - 1`. // Please fix the version when release. -v0.11.0 +v0.11.1 From 81495b57318be65efa5a027f334c4129f9e2ecc0 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 8 Nov 2023 00:04:08 +0800 Subject: [PATCH 295/348] Fix deadlink to the KEYS file in the release process (#1127) --- docs/release-process.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/release-process.md b/docs/release-process.md index e3efe8edc7..0832f17a10 100644 --- a/docs/release-process.md +++ b/docs/release-process.md @@ -114,7 +114,7 @@ It fixes the following issues: /~https://github.com/apache/pulsar-client-go/milestone/1?closed=1 Pulsar Client Go's KEYS file contains PGP keys we used to sign this release: -https://dist.apache.org/repos/dist/dev/pulsar/KEYS +https://downloads.apache.org/pulsar/KEYS Please download these packages and review this release candidate: - Review release notes From f42fdde157ce7d5a0d329e23b5b22e97044e4127 Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Fri, 10 Nov 2023 18:18:43 +0800 Subject: [PATCH 296/348] [Improve] pool sendRequest to improve producer perf (#1126) ### Motivation `sendRequest ` in producer is a frequently allocated struct, pool it can decrease the memory allocation. ### Modifications 1. Init a sync.Pool; 2. Get sendRequest from the pool when we need; 3. Reset sendRequest and put it back into the pool when it is done. --------- Co-authored-by: gunli --- pulsar/producer_partition.go | 27 ++++++++++++++++++++++++--- 1 file changed, 24 insertions(+), 3 deletions(-) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 8608df3d98..9c7c1c09d8 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -60,11 +60,20 @@ var ( errProducerClosed = newError(ProducerClosed, "producer already been closed") errMemoryBufferIsFull = newError(ClientMemoryBufferIsFull, "client memory buffer is full") - buffersPool sync.Pool + buffersPool sync.Pool + sendRequestPool *sync.Pool ) var errTopicNotFount = "TopicNotFound" +func init() { + sendRequestPool = &sync.Pool{ + New: func() interface{} { + return &sendRequest{} + }, + } +} + type partitionProducer struct { state uAtomic.Int32 client *client @@ -527,7 +536,9 @@ func (p *partitionProducer) internalSend(sr *sendRequest) { } // update chunk id sr.mm.ChunkId = proto.Int32(int32(chunkID)) - nsr := &sendRequest{ + nsr := sendRequestPool.Get().(*sendRequest) + *nsr = sendRequest{ + pool: sendRequestPool, ctx: sr.ctx, msg: sr.msg, producer: sr.producer, @@ -1150,7 +1161,9 @@ func (p *partitionProducer) internalSendAsync( return } - sr := &sendRequest{ + sr := sendRequestPool.Get().(*sendRequest) + *sr = sendRequest{ + pool: sendRequestPool, ctx: ctx, msg: msg, producer: p, @@ -1395,6 +1408,7 @@ func (p *partitionProducer) Close() { } type sendRequest struct { + pool *sync.Pool ctx context.Context msg *ProducerMessage producer *partitionProducer @@ -1477,6 +1491,13 @@ func (sr *sendRequest) done(msgID MessageID, err error) { sr.transaction.endSendOrAckOp(err) } } + + pool := sr.pool + if pool != nil { + // reset all the fields + *sr = sendRequest{} + pool.Put(sr) + } } func (p *partitionProducer) blockIfQueueFull() bool { From c91a800d3ac7dee1f00f33641895dd2ebf418311 Mon Sep 17 00:00:00 2001 From: Oliver Muir Date: Wed, 15 Nov 2023 10:47:36 +0000 Subject: [PATCH 297/348] [fix][TableView] Correct argument order to Errorf in TableView message handling(#1130) ### Motivation When TableView fails to read a message with the schema the error message is malformed, e.g. ``` ERRO[0000] msg.GetSchemaValue() failed with %!w(*pulsar.message=&{[..snipped..]}); msg is schema not found for topic: [..snipped..], schema version : [ [0 0 0 0 0 0 0 0] ] ``` The correct formatting will help users to debug their issues. ### Modifications Switch error args to match the formatting directives and context of the message. Signed-off-by: Oliver Muir Co-authored-by: Oliver Muir --- pulsar/table_view_impl.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar/table_view_impl.go b/pulsar/table_view_impl.go index 85499f193b..47f8c6c01f 100644 --- a/pulsar/table_view_impl.go +++ b/pulsar/table_view_impl.go @@ -251,7 +251,7 @@ func (tv *TableViewImpl) handleMessage(msg Message) { } else { payload = reflect.Indirect(reflect.New(tv.options.SchemaValueType)).Interface() if err := msg.GetSchemaValue(&payload); err != nil { - tv.logger.Errorf("msg.GetSchemaValue() failed with %w; msg is %v", msg, err) + tv.logger.Errorf("msg.GetSchemaValue() failed with %w; msg is %v", err, msg) } tv.data[msg.Key()] = payload } From ef0ba67444d6e92d41703af37981d075b4eb982c Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Wed, 15 Nov 2023 20:00:08 +0800 Subject: [PATCH 298/348] fix: double check before producer reconnect (#1131) Co-authored-by: gunli --- pulsar/producer_partition.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 9c7c1c09d8..f076eefe56 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -424,6 +424,14 @@ func (p *partitionProducer) reconnectToBroker() { } p.log.Info("Reconnecting to broker in ", delayReconnectTime) time.Sleep(delayReconnectTime) + + // double check + if p.getProducerState() != producerReady { + // Producer is already closing + p.log.Info("producer state not ready, exit reconnect") + return + } + atomic.AddUint64(&p.epoch, 1) err := p.grabCnx() if err == nil { From ec846ff012d425770e0a88755b0aa883ad63cbd8 Mon Sep 17 00:00:00 2001 From: Prashant Kumar <65131575+pkumar-singh@users.noreply.github.com> Date: Wed, 15 Nov 2023 18:16:15 -0800 Subject: [PATCH 299/348] [fix] When topic is terminated. Client must not retry connecting. Pending messages should be failed (#1128) ### Motivation GoLang Pulsar client library has no support for Topic termination. When a topic is terminated following should happen at client library side. 1. Producers should stop reconnecting. As once topic is terminated, it is permanent. 2. All the pending messages should be failed. ### Modifications If reconnect is failing with TopicTerminated error. Run through the pending messages queue and complete the callback. After that exit the reconnect loop and set producer state as closing. Marking producer state producerClosing will ensure that new messages are immediately failed. Co-authored-by: Prashant Kumar --- pulsar/producer_partition.go | 21 ++++++++++++++ pulsar/producer_test.go | 56 ++++++++++++++++++++++++++++++++++++ 2 files changed, 77 insertions(+) diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index f076eefe56..65eef5b66b 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -447,6 +447,27 @@ func (p *partitionProducer) reconnectToBroker() { break } + if strings.Contains(errMsg, "TopicTerminatedError") { + p.log.Info("Topic was terminated, failing pending messages, will not reconnect") + pendingItems := p.pendingQueue.ReadableSlice() + for _, item := range pendingItems { + pi := item.(*pendingItem) + if pi != nil { + pi.Lock() + requests := pi.sendRequests + for _, req := range requests { + sr := req.(*sendRequest) + if sr != nil { + sr.done(nil, newError(TopicTerminated, err.Error())) + } + } + pi.Unlock() + } + } + p.setProducerState(producerClosing) + break + } + if maxRetry > 0 { maxRetry-- } diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 49e225f3e1..f30ae65fcf 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -1158,6 +1158,62 @@ func TestFailedSchemaEncode(t *testing.T) { wg.Wait() } +func TestTopicTermination(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: serviceURL, + }) + assert.NoError(t, err) + defer client.Close() + + topicName := newTopicName() + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topicName, + SubscriptionName: "send_timeout_sub", + }) + assert.Nil(t, err) + defer consumer.Close() // subscribe but do nothing + + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topicName, + SendTimeout: 2 * time.Second, + }) + assert.Nil(t, err) + defer producer.Close() + + afterCh := time.After(5 * time.Second) + terminatedChan := make(chan bool) + go func() { + for { + _, err := producer.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 1024), + }) + if err != nil { + e := err.(*Error) + if e.result == TopicTerminated { + terminatedChan <- true + } else { + terminatedChan <- false + } + } + time.Sleep(1 * time.Millisecond) + } + }() + + terminateURL := adminURL + "/admin/v2/persistent/public/default/" + topicName + "/terminate" + log.Info(terminateURL) + makeHTTPCall(t, http.MethodPost, terminateURL, "") + + for { + select { + case d := <-terminatedChan: + assert.Equal(t, d, true) + return + case <-afterCh: + assert.Fail(t, "Time is up. Topic should have been terminated by now") + } + } +} + func TestSendTimeout(t *testing.T) { quotaURL := adminURL + "/admin/v2/namespaces/public/default/backlogQuota" quotaFmt := `{"limit": "%d", "policy": "producer_request_hold"}` From d457442434c33f0fef7469c24d25de04bc718aa6 Mon Sep 17 00:00:00 2001 From: Oliver Muir Date: Mon, 20 Nov 2023 09:41:50 +0000 Subject: [PATCH 300/348] [Issue 1132] Fix JSONSchema unmarshalling in TableView (#1133) Fixes #1132 ### Motivation Fix issue #1132 - using JSONSchema with TableView ### Modifications - Set a concrete type in the `payload` variable before JSON-unmarshalling into that variable. This allows the JSON package to identify and use the type rather than seeing it as `interface{}`. - Use `reflect.Indirect(payload).Interface()` when storing the payload and passing it to listeners to remove the pointer from `reflect.New`. - Add test coverage for `TableView.Get` covering all supported schema types. - Add test coverage for `TableView.ForEachAndListen` for JSONSchema. Additional minor changes. They didn't seem worth their own MRs but I'm happy to split them out if that's better. - Correct typo in comments on `TableView.ForEach` and `TableView.ForEachAndListen` interface methods. - Correct `TableView.ForEachAndListen` comment to clarify that it continues to call the given action on future messages. - Correct formatting directive (`%w` -> `%v`) in error log `tv.logger.Errorf("msg.GetSchemaValue() failed with %v; msg is %v", err, msg)`. (This indirectly calls `fmt.Sprintf` in logrus which doesn't support `%w`). --- pulsar/table_view.go | 6 +- pulsar/table_view_impl.go | 11 +- pulsar/table_view_test.go | 211 ++++++++++++++++++++++++++++++++++++++ 3 files changed, 219 insertions(+), 9 deletions(-) diff --git a/pulsar/table_view.go b/pulsar/table_view.go index e566bf0bec..58a664aed7 100644 --- a/pulsar/table_view.go +++ b/pulsar/table_view.go @@ -65,12 +65,12 @@ type TableView interface { // Keys returns a slice of the keys contained in this TableView. Keys() []string - // ForEach performs the give action for each entry in this map until all entries have been processed or the action + // ForEach performs the given action for each entry in this map until all entries have been processed or the action // returns an error. ForEach(func(string, interface{}) error) error - // ForEachAndListen performs the give action for each entry in this map until all entries have been processed or - // the action returns an error. + // ForEachAndListen performs the given action for each entry in this map until all entries have been processed or + // the action returns an error. The given action will then be performed on each new entry in this map. ForEachAndListen(func(string, interface{}) error) error // Close closes the table view and releases resources allocated. diff --git a/pulsar/table_view_impl.go b/pulsar/table_view_impl.go index 47f8c6c01f..17e0b90f3b 100644 --- a/pulsar/table_view_impl.go +++ b/pulsar/table_view_impl.go @@ -245,19 +245,18 @@ func (tv *TableViewImpl) handleMessage(msg Message) { tv.dataMu.Lock() defer tv.dataMu.Unlock() - var payload interface{} + payload := reflect.New(tv.options.SchemaValueType) if len(msg.Payload()) == 0 { delete(tv.data, msg.Key()) } else { - payload = reflect.Indirect(reflect.New(tv.options.SchemaValueType)).Interface() - if err := msg.GetSchemaValue(&payload); err != nil { - tv.logger.Errorf("msg.GetSchemaValue() failed with %w; msg is %v", err, msg) + if err := msg.GetSchemaValue(payload.Interface()); err != nil { + tv.logger.Errorf("msg.GetSchemaValue() failed with %v; msg is %v", err, msg) } - tv.data[msg.Key()] = payload + tv.data[msg.Key()] = reflect.Indirect(payload).Interface() } for _, listener := range tv.listeners { - if err := listener(msg.Key(), payload); err != nil { + if err := listener(msg.Key(), reflect.Indirect(payload).Interface()); err != nil { tv.logger.Errorf("table view listener failed for %v: %w", msg, err) } } diff --git a/pulsar/table_view_test.go b/pulsar/table_view_test.go index d29b24d298..45b9441169 100644 --- a/pulsar/table_view_test.go +++ b/pulsar/table_view_test.go @@ -24,6 +24,7 @@ import ( "testing" "time" + pb "github.com/apache/pulsar-client-go/integration-tests/pb" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -80,6 +81,157 @@ func TestTableView(t *testing.T) { } } +func TestTableViewSchemas(t *testing.T) { + var tests = []struct { + name string + schema Schema + schemaType interface{} + producerValue interface{} + expValueOut interface{} + valueCheck func(t *testing.T, got interface{}) // Overrides expValueOut for more complex checks + }{ + { + name: "StringSchema", + schema: NewStringSchema(nil), + schemaType: strPointer("hello pulsar"), + producerValue: "hello pulsar", + expValueOut: strPointer("hello pulsar"), + }, + { + name: "JSONSchema", + schema: NewJSONSchema(exampleSchemaDef, nil), + schemaType: testJSON{}, + producerValue: testJSON{ID: 1, Name: "Pulsar"}, + expValueOut: testJSON{ID: 1, Name: "Pulsar"}, + }, + { + name: "JSONSchema pointer type", + schema: NewJSONSchema(exampleSchemaDef, nil), + schemaType: &testJSON{ID: 1, Name: "Pulsar"}, + producerValue: testJSON{ID: 1, Name: "Pulsar"}, + expValueOut: &testJSON{ID: 1, Name: "Pulsar"}, + }, + { + name: "AvroSchema", + schema: NewAvroSchema(exampleSchemaDef, nil), + schemaType: testAvro{ID: 1, Name: "Pulsar"}, + producerValue: testAvro{ID: 1, Name: "Pulsar"}, + expValueOut: testAvro{ID: 1, Name: "Pulsar"}, + }, + { + name: "Int8Schema", + schema: NewInt8Schema(nil), + schemaType: int8(0), + producerValue: int8(1), + expValueOut: int8(1), + }, + { + name: "Int16Schema", + schema: NewInt16Schema(nil), + schemaType: int16(0), + producerValue: int16(1), + expValueOut: int16(1), + }, + { + name: "Int32Schema", + schema: NewInt32Schema(nil), + schemaType: int32(0), + producerValue: int32(1), + expValueOut: int32(1), + }, + { + name: "Int64Schema", + schema: NewInt64Schema(nil), + schemaType: int64(0), + producerValue: int64(1), + expValueOut: int64(1), + }, + { + name: "FloatSchema", + schema: NewFloatSchema(nil), + schemaType: float32(0), + producerValue: float32(1), + expValueOut: float32(1), + }, + { + name: "DoubleSchema", + schema: NewDoubleSchema(nil), + schemaType: float64(0), + producerValue: float64(1), + expValueOut: float64(1), + }, + { + name: "ProtoSchema", + schema: NewProtoSchema(protoSchemaDef, nil), + schemaType: pb.Test{}, + producerValue: &pb.Test{Num: 1, Msf: "Pulsar"}, + valueCheck: func(t *testing.T, got interface{}) { + assert.IsType(t, pb.Test{}, got) + assert.Equal(t, int32(1), got.(pb.Test).Num) + assert.Equal(t, "Pulsar", got.(pb.Test).Msf) + }, + }, + { + name: "ProtoNativeSchema", + schema: NewProtoNativeSchemaWithMessage(&pb.Test{}, nil), + schemaType: pb.Test{}, + producerValue: &pb.Test{Num: 1, Msf: "Pulsar"}, + valueCheck: func(t *testing.T, got interface{}) { + assert.IsType(t, pb.Test{}, got) + assert.Equal(t, int32(1), got.(pb.Test).Num) + assert.Equal(t, "Pulsar", got.(pb.Test).Msf) + }, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.NoError(t, err) + defer client.Close() + + topic := newTopicName() + + // create producer + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + Schema: test.schema, + }) + assert.NoError(t, err) + defer producer.Close() + + _, err = producer.Send(context.Background(), &ProducerMessage{ + Key: "testKey", + Value: test.producerValue, + }) + assert.NoError(t, err) + + // create table view + tv, err := client.CreateTableView(TableViewOptions{ + Topic: topic, + Schema: test.schema, + SchemaValueType: reflect.TypeOf(test.schemaType), + }) + assert.NoError(t, err) + defer tv.Close() + + value := tv.Get("testKey") + if test.valueCheck != nil { + test.valueCheck(t, value) + } else { + assert.IsType(t, test.expValueOut, value) + assert.Equal(t, test.expValueOut, value) + } + }) + } +} + +func strPointer(s string) *string { + return &s +} + func TestPublishNilValue(t *testing.T) { client, err := NewClient(ClientOptions{ URL: lookupURL, @@ -143,3 +295,62 @@ func TestPublishNilValue(t *testing.T) { assert.Equal(t, *(tv.Get("key-2").(*string)), "value-2") } + +func TestForEachAndListenJSONSchema(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.NoError(t, err) + defer client.Close() + + topic := newTopicName() + schema := NewJSONSchema(exampleSchemaDef, nil) + + // create table view + tv, err := client.CreateTableView(TableViewOptions{ + Topic: topic, + Schema: schema, + SchemaValueType: reflect.TypeOf(testJSON{}), + }) + assert.NoError(t, err) + defer tv.Close() + + // create listener + valuePrefix := "hello pulsar: " + tv.ForEachAndListen(func(key string, value interface{}) error { + t.Log("foreach" + key) + s, ok := value.(testJSON) + assert.Truef(t, ok, "expected value to be testJSON type got %T", value) + assert.Equal(t, fmt.Sprintf(valuePrefix+key), s.Name) + return nil + }) + + // create producer + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + Schema: schema, + }) + assert.NoError(t, err) + defer producer.Close() + + numMsg := 10 + for i := 0; i < numMsg; i++ { + key := fmt.Sprintf("%d", i) + t.Log("producing" + key) + _, err = producer.Send(context.Background(), &ProducerMessage{ + Key: key, + Value: testJSON{ + ID: i, + Name: fmt.Sprintf(valuePrefix + key), + }, + }) + assert.NoError(t, err) + } + + // Wait until tv receives all messages + for tv.Size() < 10 { + time.Sleep(time.Second * 1) + t.Logf("TableView number of elements: %d", tv.Size()) + } +} From 067dca1f47e6aab9e65a24f8796aa09ca48c2f4a Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Fri, 24 Nov 2023 16:21:12 +0800 Subject: [PATCH 301/348] [Improve] Set dlq producerName (#1137) ### Motivation To keep consistent with the Java client. Releted PR: /~https://github.com/apache/pulsar/pull/21589 *Explain here the context, and why you're making that change. What is the problem you're trying to solve.* ### Modifications Set DLQ producerName `%s-%s-DLQ` --- pulsar/consumer_impl.go | 2 +- pulsar/consumer_regex_test.go | 4 ++-- pulsar/consumer_test.go | 6 +++++- pulsar/dlq_router.go | 29 +++++++++++++++++++---------- pulsar/reader_impl.go | 2 +- 5 files changed, 28 insertions(+), 15 deletions(-) diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index d782beab5e..d701ab16d6 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -167,7 +167,7 @@ func newConsumer(client *client, options ConsumerOptions) (Consumer, error) { } } - dlq, err := newDlqRouter(client, options.DLQ, client.log) + dlq, err := newDlqRouter(client, options.DLQ, options.Topic, options.SubscriptionName, client.log) if err != nil { return nil, err } diff --git a/pulsar/consumer_regex_test.go b/pulsar/consumer_regex_test.go index 9cb600fe3a..de9461341f 100644 --- a/pulsar/consumer_regex_test.go +++ b/pulsar/consumer_regex_test.go @@ -154,7 +154,7 @@ func runRegexConsumerDiscoverPatternAll(t *testing.T, c Client, namespace string AutoDiscoveryPeriod: 5 * time.Minute, } - dlq, _ := newDlqRouter(c.(*client), nil, log.DefaultNopLogger()) + dlq, _ := newDlqRouter(c.(*client), nil, tn.Topic, "regex-sub", log.DefaultNopLogger()) rlq, _ := newRetryRouter(c.(*client), nil, false, log.DefaultNopLogger()) consumer, err := newRegexConsumer(c.(*client), opts, tn, pattern, make(chan ConsumerMessage, 1), dlq, rlq) if err != nil { @@ -192,7 +192,7 @@ func runRegexConsumerDiscoverPatternFoo(t *testing.T, c Client, namespace string AutoDiscoveryPeriod: 5 * time.Minute, } - dlq, _ := newDlqRouter(c.(*client), nil, log.DefaultNopLogger()) + dlq, _ := newDlqRouter(c.(*client), nil, tn.Topic, "regex-sub", log.DefaultNopLogger()) rlq, _ := newRetryRouter(c.(*client), nil, false, log.DefaultNopLogger()) consumer, err := newRegexConsumer(c.(*client), opts, tn, pattern, make(chan ConsumerMessage, 1), dlq, rlq) if err != nil { diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 2d785b4b23..33382cff99 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -1449,9 +1449,10 @@ func DLQWithProducerOptions(t *testing.T, prodOpt *ProducerOptions) { if prodOpt != nil { dlqPolicy.ProducerOptions = *prodOpt } + sub := "my-sub" consumer, err := client.Subscribe(ConsumerOptions{ Topic: topic, - SubscriptionName: "my-sub", + SubscriptionName: sub, NackRedeliveryDelay: 1 * time.Second, Type: Shared, DLQ: &dlqPolicy, @@ -1506,6 +1507,9 @@ func DLQWithProducerOptions(t *testing.T, prodOpt *ProducerOptions) { expectMsg := fmt.Sprintf("hello-%d", expectedMsgIdx) assert.Equal(t, []byte(expectMsg), msg.Payload()) + // check dql produceName + assert.Equal(t, msg.ProducerName(), fmt.Sprintf("%s-%s-DLQ", topic, sub)) + // check original messageId assert.NotEmpty(t, msg.Properties()[PropertyOriginMessageID]) diff --git a/pulsar/dlq_router.go b/pulsar/dlq_router.go index 6d30003640..5b9314bddc 100644 --- a/pulsar/dlq_router.go +++ b/pulsar/dlq_router.go @@ -19,6 +19,7 @@ package pulsar import ( "context" + "fmt" "time" "github.com/apache/pulsar-client-go/pulsar/internal" @@ -26,19 +27,24 @@ import ( ) type dlqRouter struct { - client Client - producer Producer - policy *DLQPolicy - messageCh chan ConsumerMessage - closeCh chan interface{} - log log.Logger + client Client + producer Producer + policy *DLQPolicy + messageCh chan ConsumerMessage + closeCh chan interface{} + topicName string + subscriptionName string + log log.Logger } -func newDlqRouter(client Client, policy *DLQPolicy, logger log.Logger) (*dlqRouter, error) { +func newDlqRouter(client Client, policy *DLQPolicy, topicName, subscriptionName string, + logger log.Logger) (*dlqRouter, error) { r := &dlqRouter{ - client: client, - policy: policy, - log: logger, + client: client, + policy: policy, + topicName: topicName, + subscriptionName: subscriptionName, + log: logger, } if policy != nil { @@ -152,6 +158,9 @@ func (r *dlqRouter) getProducer(schema Schema) Producer { opt := r.policy.ProducerOptions opt.Topic = r.policy.DeadLetterTopic opt.Schema = schema + if opt.Name == "" { + opt.Name = fmt.Sprintf("%s-%s-DLQ", r.topicName, r.subscriptionName) + } // the origin code sets to LZ4 compression with no options // so the new design allows compression type to be overwritten but still set lz4 by default diff --git a/pulsar/reader_impl.go b/pulsar/reader_impl.go index ffc92dedde..0999e88fee 100644 --- a/pulsar/reader_impl.go +++ b/pulsar/reader_impl.go @@ -127,7 +127,7 @@ func newReader(client *client, options ReaderOptions) (Reader, error) { } // Provide dummy dlq router with not dlq policy - dlq, err := newDlqRouter(client, nil, client.log) + dlq, err := newDlqRouter(client, nil, options.Topic, options.SubscriptionName, client.log) if err != nil { return nil, err } From 72aed95ae17baa4aa5ad8db7fb4abbf96f6c1c59 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gon=C3=A7alo=20Rodrigues?= Date: Fri, 8 Dec 2023 10:53:41 +0100 Subject: [PATCH 302/348] [fix] fix channel deadlock in regexp consumer (#1141) ### Motivation When using a regexp consumer, there's a race condition between producing and consuming new discovered topics. If the discover topic takes too long or the auto discovery period is too short, then multiple ticker.C messages may be processed in a row which will block on the subcsribe/unsubscribe channels as they only have a buffer size of 1. This will block new topics from being discovered forever. ### Modifications Moved the consumers into their own goroutines and use an unbuffered channel. There's multiple ways to go about it but it's good practice to keep consumers and producers separate. Consumers are run until the channels they are consumed from are closed, which happens when the producer (monitor) returns. --- pulsar/consumer_regex.go | 31 ++++++----------- pulsar/consumer_regex_test.go | 63 +++++++++++++++++++++++++++++++++++ 2 files changed, 74 insertions(+), 20 deletions(-) diff --git a/pulsar/consumer_regex.go b/pulsar/consumer_regex.go index 79e2293b49..d36694ef90 100644 --- a/pulsar/consumer_regex.go +++ b/pulsar/consumer_regex.go @@ -50,8 +50,6 @@ type regexConsumer struct { consumersLock sync.Mutex consumers map[string]Consumer - subscribeCh chan []string - unsubscribeCh chan []string closeOnce sync.Once closeCh chan struct{} @@ -75,9 +73,7 @@ func newRegexConsumer(c *client, opts ConsumerOptions, tn *internal.TopicName, p namespace: tn.Namespace, pattern: pattern, - consumers: make(map[string]Consumer), - subscribeCh: make(chan []string, 1), - unsubscribeCh: make(chan []string, 1), + consumers: make(map[string]Consumer), closeCh: make(chan struct{}), @@ -163,12 +159,11 @@ func (c *regexConsumer) Ack(msg Message) error { return c.AckID(msg.ID()) } -func (c *regexConsumer) ReconsumeLater(msg Message, delay time.Duration) { +func (c *regexConsumer) ReconsumeLater(_ Message, _ time.Duration) { c.log.Warnf("regexp consumer not support ReconsumeLater yet.") } -func (c *regexConsumer) ReconsumeLaterWithCustomProperties(msg Message, customProperties map[string]string, - delay time.Duration) { +func (c *regexConsumer) ReconsumeLaterWithCustomProperties(_ Message, _ map[string]string, _ time.Duration) { c.log.Warnf("regexp consumer not support ReconsumeLaterWithCustomProperties yet.") } @@ -297,11 +292,11 @@ func (c *regexConsumer) Close() { }) } -func (c *regexConsumer) Seek(msgID MessageID) error { +func (c *regexConsumer) Seek(_ MessageID) error { return newError(SeekFailed, "seek command not allowed for regex consumer") } -func (c *regexConsumer) SeekByTime(time time.Time) error { +func (c *regexConsumer) SeekByTime(_ time.Time) error { return newError(SeekFailed, "seek command not allowed for regex consumer") } @@ -329,14 +324,6 @@ func (c *regexConsumer) monitor() { if !c.closed() { c.discover() } - case topics := <-c.subscribeCh: - if len(topics) > 0 && !c.closed() { - c.subscribe(topics, c.dlq, c.rlq) - } - case topics := <-c.unsubscribeCh: - if len(topics) > 0 && !c.closed() { - c.unsubscribe(topics) - } } } } @@ -358,8 +345,12 @@ func (c *regexConsumer) discover() { }). Debug("discover topics") - c.unsubscribeCh <- staleTopics - c.subscribeCh <- newTopics + if len(staleTopics) > 0 { + c.unsubscribe(staleTopics) + } + if len(newTopics) > 0 { + c.subscribe(newTopics, c.dlq, c.rlq) + } } func (c *regexConsumer) knownTopics() []string { diff --git a/pulsar/consumer_regex_test.go b/pulsar/consumer_regex_test.go index de9461341f..3e5f1d61db 100644 --- a/pulsar/consumer_regex_test.go +++ b/pulsar/consumer_regex_test.go @@ -241,6 +241,7 @@ func runRegexConsumerDiscoverPatternFoo(t *testing.T, c Client, namespace string func TestRegexConsumer(t *testing.T) { t.Run("MatchOneTopic", runWithClientNamespace(runRegexConsumerMatchOneTopic)) t.Run("AddTopic", runWithClientNamespace(runRegexConsumerAddMatchingTopic)) + t.Run("AutoDiscoverTopics", runWithClientNamespace(runRegexConsumerAutoDiscoverTopics)) } func runRegexConsumerMatchOneTopic(t *testing.T, c Client, namespace string) { @@ -346,6 +347,68 @@ func runRegexConsumerAddMatchingTopic(t *testing.T, c Client, namespace string) } } +func runRegexConsumerAutoDiscoverTopics(t *testing.T, c Client, namespace string) { + topicsPattern := fmt.Sprintf("persistent://%s/foo.*", namespace) + opts := ConsumerOptions{ + TopicsPattern: topicsPattern, + SubscriptionName: "regex-sub", + // this is purposefully short to test parallelism between discover and subscribe calls + AutoDiscoveryPeriod: 1 * time.Nanosecond, + } + consumer, err := c.Subscribe(opts) + if err != nil { + t.Fatal(err) + } + defer consumer.Close() + + topicInRegex1 := namespace + "/foo-topic-1" + p1, err := c.CreateProducer(ProducerOptions{ + Topic: topicInRegex1, + DisableBatching: true, + }) + if err != nil { + t.Fatal(err) + } + defer p1.Close() + + topicInRegex2 := namespace + "/foo-topic-2" + p2, err := c.CreateProducer(ProducerOptions{ + Topic: topicInRegex2, + DisableBatching: true, + }) + if err != nil { + t.Fatal(err) + } + defer p2.Close() + + time.Sleep(100 * time.Millisecond) + + err = genMessages(p1, 5, func(idx int) string { + return fmt.Sprintf("foo-message-%d", idx) + }) + if err != nil { + t.Fatal(err) + } + + err = genMessages(p2, 5, func(idx int) string { + return fmt.Sprintf("foo-message-%d", idx) + }) + if err != nil { + t.Fatal(err) + } + + ctx := context.Background() + for i := 0; i < 10; i++ { + m, err := consumer.Receive(ctx) + if err != nil { + t.Errorf("failed to receive message error: %+v", err) + } else { + assert.Truef(t, strings.HasPrefix(string(m.Payload()), "foo-"), + "message does not start with foo: %s", string(m.Payload())) + } + } +} + func genMessages(p Producer, num int, msgFn func(idx int) string) error { ctx := context.Background() for i := 0; i < num; i++ { From bd11581867c88f93e4b6f247d82914f9eb4ee476 Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Fri, 8 Dec 2023 17:54:35 +0800 Subject: [PATCH 303/348] [Fix][Producer] handle TopicNotFound/TopicTerminated/ProducerBlockedQuotaExceededException/ProducerFenced when reconnecting (#1134) Master Issue: #1128 ### Motivation In Java client, when we get TopicNotFound/TopicTerminated/ProducerBlockedQuotaExceededException/ProducerFenced, we should failPendingMessages, and close producer. But in Go client, we forget to handle ProducerBlockedQuotaExceededException/ProducerFenced, and in #1128, we just call sr.done(), actually we should call failPendingMessages(). /~https://github.com/apache/pulsar-client-go/pull/1128/files /~https://github.com/apache/pulsar/blob/master/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L1663 ### Modifications 1. rename `errMsgTopicNotFount` to `errMsgTopicNotFound` 2. handle TopicTerminated/ProducerBlockedQuotaExceededException/ProducerFenced, call `failPendingMessages()`; --------- Co-authored-by: gunli --- pulsar/consumer_partition.go | 2 +- pulsar/error.go | 6 +++ pulsar/producer_partition.go | 74 ++++++++++++++++++++++-------------- pulsar/producer_test.go | 13 ++++--- 4 files changed, 60 insertions(+), 35 deletions(-) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 2ba842744c..fd6441c1cb 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -1659,7 +1659,7 @@ func (pc *partitionConsumer) reconnectToBroker() { } pc.log.WithError(err).Error("Failed to create consumer at reconnect") errMsg := err.Error() - if strings.Contains(errMsg, errTopicNotFount) { + if strings.Contains(errMsg, errMsgTopicNotFound) { // when topic is deleted, we should give up reconnection. pc.log.Warn("Topic Not Found.") break diff --git a/pulsar/error.go b/pulsar/error.go index 73a0b6067b..25498cfba4 100644 --- a/pulsar/error.go +++ b/pulsar/error.go @@ -114,6 +114,12 @@ const ( TransactionNoFoundError // ClientMemoryBufferIsFull client limit buffer is full ClientMemoryBufferIsFull + // ProducerFenced When a producer asks and fail to get exclusive producer access, + // or loses the exclusive 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 + ProducerFenced ) // Error implement error interface, composed of two parts: msg and result. diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 65eef5b66b..46167d0cf1 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -64,7 +64,12 @@ var ( sendRequestPool *sync.Pool ) -var errTopicNotFount = "TopicNotFound" +const ( + errMsgTopicNotFound = "TopicNotFound" + errMsgTopicTerminated = "TopicTerminatedError" + errMsgProducerBlockedQuotaExceededException = "ProducerBlockedQuotaExceededException" + errMsgProducerFenced = "ProducerFenced" +) func init() { sendRequestPool = &sync.Pool{ @@ -441,30 +446,28 @@ func (p *partitionProducer) reconnectToBroker() { } p.log.WithError(err).Error("Failed to create producer at reconnect") errMsg := err.Error() - if strings.Contains(errMsg, errTopicNotFount) { + if strings.Contains(errMsg, errMsgTopicNotFound) { // when topic is deleted, we should give up reconnection. - p.log.Warn("Topic Not Found.") + p.log.Warn("Topic not found, stop reconnecting, close the producer") + p.doClose(newError(TopicNotFound, err.Error())) break } - if strings.Contains(errMsg, "TopicTerminatedError") { - p.log.Info("Topic was terminated, failing pending messages, will not reconnect") - pendingItems := p.pendingQueue.ReadableSlice() - for _, item := range pendingItems { - pi := item.(*pendingItem) - if pi != nil { - pi.Lock() - requests := pi.sendRequests - for _, req := range requests { - sr := req.(*sendRequest) - if sr != nil { - sr.done(nil, newError(TopicTerminated, err.Error())) - } - } - pi.Unlock() - } - } - p.setProducerState(producerClosing) + if strings.Contains(errMsg, errMsgTopicTerminated) { + p.log.Warn("Topic was terminated, failing pending messages, stop reconnecting, close the producer") + p.doClose(newError(TopicTerminated, err.Error())) + break + } + + if strings.Contains(errMsg, errMsgProducerBlockedQuotaExceededException) { + p.log.Warn("Producer was blocked by quota exceed exception, failing pending messages, stop reconnecting") + p.failPendingMessages(newError(ProducerBlockedQuotaExceededException, err.Error())) + break + } + + if strings.Contains(errMsg, errMsgProducerFenced) { + p.log.Warn("Producer was fenced, failing pending messages, stop reconnecting") + p.doClose(newError(ProducerFenced, err.Error())) break } @@ -481,10 +484,18 @@ func (p *partitionProducer) reconnectToBroker() { func (p *partitionProducer) runEventsLoop() { for { select { - case data := <-p.dataChan: + case data, ok := <-p.dataChan: + // when doClose() is call, p.dataChan will be closed, data will be nil + if !ok { + return + } p.internalSend(data) - case i := <-p.cmdChan: - switch v := i.(type) { + case cmd, ok := <-p.cmdChan: + // when doClose() is call, p.dataChan will be closed, cmd will be nil + if !ok { + return + } + switch v := cmd.(type) { case *flushRequest: p.internalFlush(v) case *closeProducer: @@ -1321,13 +1332,18 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) func (p *partitionProducer) internalClose(req *closeProducer) { defer close(req.doneCh) + + p.doClose(errProducerClosed) +} + +func (p *partitionProducer) doClose(reason error) { if !p.casProducerState(producerReady, producerClosing) { return } + p.log.Info("Closing producer") defer close(p.dataChan) defer close(p.cmdChan) - p.log.Info("Closing producer") id := p.client.rpcClient.NewRequestID() _, err := p.client.rpcClient.RequestOnCnx(p._getConn(), id, pb.BaseCommand_CLOSE_PRODUCER, &pb.CommandCloseProducer{ @@ -1340,7 +1356,7 @@ func (p *partitionProducer) internalClose(req *closeProducer) { } else { p.log.Info("Closed producer") } - p.failPendingMessages() + p.failPendingMessages(reason) if p.batchBuilder != nil { if err = p.batchBuilder.Close(); err != nil { @@ -1353,7 +1369,7 @@ func (p *partitionProducer) internalClose(req *closeProducer) { p.batchFlushTicker.Stop() } -func (p *partitionProducer) failPendingMessages() { +func (p *partitionProducer) failPendingMessages(err error) { curViewItems := p.pendingQueue.ReadableSlice() viewSize := len(curViewItems) if viewSize <= 0 { @@ -1378,11 +1394,11 @@ func (p *partitionProducer) failPendingMessages() { for _, i := range pi.sendRequests { sr := i.(*sendRequest) - sr.done(nil, errProducerClosed) + sr.done(nil, err) } // flag the sending has completed with error, flush make no effect - pi.done(errProducerClosed) + pi.done(err) pi.Unlock() // finally reached the last view item, current iteration ends diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index f30ae65fcf..0f89069243 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -29,14 +29,16 @@ import ( "testing" "time" - "github.com/apache/pulsar-client-go/pulsar/internal" - pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" "github.com/stretchr/testify/assert" "google.golang.org/protobuf/proto" + "github.com/apache/pulsar-client-go/pulsar/internal" + pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" + + log "github.com/sirupsen/logrus" + "github.com/apache/pulsar-client-go/pulsar/crypto" plog "github.com/apache/pulsar-client-go/pulsar/log" - log "github.com/sirupsen/logrus" ) func TestInvalidURL(t *testing.T) { @@ -1168,7 +1170,7 @@ func TestTopicTermination(t *testing.T) { topicName := newTopicName() consumer, err := client.Subscribe(ConsumerOptions{ Topic: topicName, - SubscriptionName: "send_timeout_sub", + SubscriptionName: "topic_terminated_sub", }) assert.Nil(t, err) defer consumer.Close() // subscribe but do nothing @@ -1189,7 +1191,7 @@ func TestTopicTermination(t *testing.T) { }) if err != nil { e := err.(*Error) - if e.result == TopicTerminated { + if e.result == TopicTerminated || err == errProducerClosed { terminatedChan <- true } else { terminatedChan <- false @@ -1210,6 +1212,7 @@ func TestTopicTermination(t *testing.T) { return case <-afterCh: assert.Fail(t, "Time is up. Topic should have been terminated by now") + return } } } From 062fefe12be229617b1f7e499969355c02470793 Mon Sep 17 00:00:00 2001 From: Gaylor Bosson Date: Tue, 19 Dec 2023 11:17:47 +0100 Subject: [PATCH 304/348] Avoid a panic when using transaction (#1144) ### Motivation When aborting or committing a transaction, it can happen that an error returned from the client triggers a panic when attempting to cast it to a Pulsar error. ### Modifications A proper check is performed when casting the error. --- pulsar/transaction_impl.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar/transaction_impl.go b/pulsar/transaction_impl.go index 8a24c46429..2eb8aca9a8 100644 --- a/pulsar/transaction_impl.go +++ b/pulsar/transaction_impl.go @@ -101,7 +101,7 @@ func (txn *transaction) Commit(ctx context.Context) error { if err == nil { atomic.StoreInt32((*int32)(&txn.state), int32(TxnCommitted)) } else { - if err.(*Error).Result() == TransactionNoFoundError || err.(*Error).Result() == InvalidStatus { + if e, ok := err.(*Error); ok && (e.Result() == TransactionNoFoundError || e.Result() == InvalidStatus) { atomic.StoreInt32((*int32)(&txn.state), int32(TxnError)) return err } @@ -127,7 +127,7 @@ func (txn *transaction) Abort(ctx context.Context) error { if err == nil { atomic.StoreInt32((*int32)(&txn.state), int32(TxnAborted)) } else { - if err.(*Error).Result() == TransactionNoFoundError || err.(*Error).Result() == InvalidStatus { + if e, ok := err.(*Error); ok && (e.Result() == TransactionNoFoundError || e.Result() == InvalidStatus) { atomic.StoreInt32((*int32)(&txn.state), int32(TxnError)) } else { txn.opsFlow <- true From 443072bf5151283e412fd9191c6efdc527bc5816 Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Wed, 27 Dec 2023 17:32:25 +0800 Subject: [PATCH 305/348] [Improve] Update connection.lastDataReceivedTime when it is ready (#1145) Co-authored-by: gunli --- pulsar/internal/connection.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index 840ecc4f41..cb8c680d48 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -338,6 +338,7 @@ func (c *connection) doHandshake() bool { c.maxMessageSize = MaxMessageSize } c.log.Info("Connection is ready") + c.setLastDataReceived(time.Now()) c.changeState(connectionReady) return true } From f491e09e025ad72806f2f49b4a195b3c5b27d55a Mon Sep 17 00:00:00 2001 From: gunli <24350715@qq.com> Date: Fri, 29 Dec 2023 10:57:27 +0800 Subject: [PATCH 306/348] [Improve][Producer] normalize and export the errors (#1143) * [Improve][Producer] normalize and export the errors * update schema error * update go version to 1.20 to support errors.Join() * use errors.Is() to test an error * use github.com/hashicorp/go-multierror to join errors instead of errors.Join() of Go 1.20 * revert go version to 1.18 * revert go version to 1.18 * update ErrSchema according to the CR sugguestions * update ErrTransaction to a normal error * rename ErrProducerBlocked to ErrProducerBlockedQuotaExceeded * add license header * fix unit test error --------- Co-authored-by: gunli --- go.mod | 2 + go.sum | 4 ++ pulsar/consumer_test.go | 2 +- pulsar/error.go | 8 ++++ pulsar/error_test.go | 36 ++++++++++++++++ pulsar/producer_partition.go | 84 ++++++++++++++++++++---------------- pulsar/producer_test.go | 15 +++---- 7 files changed, 104 insertions(+), 47 deletions(-) create mode 100644 pulsar/error_test.go diff --git a/go.mod b/go.mod index f88d6ad576..52bfe43a96 100644 --- a/go.mod +++ b/go.mod @@ -43,6 +43,8 @@ require ( github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect github.com/golang/snappy v0.0.1 // indirect github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect + github.com/hashicorp/errwrap v1.0.0 // indirect + github.com/hashicorp/go-multierror v1.1.1 // indirect github.com/inconshreveable/mousetrap v1.0.1 // indirect github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect diff --git a/go.sum b/go.sum index 00a44917d8..e8a9e76be7 100644 --- a/go.sum +++ b/go.sum @@ -160,6 +160,10 @@ github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5m github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c h1:6rhixN/i8ZofjG1Y75iExal34USq5p+wiN1tpie8IrU= github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c/go.mod h1:NMPJylDgVpX0MLRlPy15sqSwOFv/U1GZ2m21JhFfek0= +github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= +github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= +github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 33382cff99..8b983d0d19 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -4299,7 +4299,7 @@ func TestConsumerMemoryLimit(t *testing.T) { Payload: createTestMessagePayload(1), }) // Producer can't send message - assert.Equal(t, true, errors.Is(err, errMemoryBufferIsFull)) + assert.Equal(t, true, errors.Is(err, ErrMemoryBufferIsFull)) } func TestMultiConsumerMemoryLimit(t *testing.T) { diff --git a/pulsar/error.go b/pulsar/error.go index 25498cfba4..f03799342a 100644 --- a/pulsar/error.go +++ b/pulsar/error.go @@ -21,6 +21,7 @@ import ( "fmt" proto "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" + "github.com/hashicorp/go-multierror" ) // Result used to represent pulsar processing is an alias of type int. @@ -245,3 +246,10 @@ func getErrorFromServerError(serverError *proto.ServerError) error { return newError(UnknownError, serverError.String()) } } + +// joinErrors can join multiple errors into one error, and the returned error can be tested by errors.Is() +// we use github.com/hashicorp/go-multierror instead of errors.Join() of Go 1.20 so that we can compile pulsar +// go client with go versions that newer than go 1.13 +func joinErrors(errs ...error) error { + return multierror.Append(nil, errs...) +} diff --git a/pulsar/error_test.go b/pulsar/error_test.go new file mode 100644 index 0000000000..5403effc2d --- /dev/null +++ b/pulsar/error_test.go @@ -0,0 +1,36 @@ +// 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 pulsar + +import ( + "errors" + "testing" + + "github.com/stretchr/testify/assert" +) + +func Test_joinErrors(t *testing.T) { + err1 := errors.New("err1") + err2 := errors.New("err2") + err3 := errors.New("err3") + err := joinErrors(ErrInvalidMessage, err1, err2) + assert.True(t, errors.Is(err, ErrInvalidMessage)) + assert.True(t, errors.Is(err, err1)) + assert.True(t, errors.Is(err, err2)) + assert.False(t, errors.Is(err, err3)) +} diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 46167d0cf1..1b79053e38 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -51,14 +51,21 @@ const ( ) 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") - errMetaTooLarge = newError(InvalidMessage, "message metadata size exceeds MaxMessageSize") - errProducerClosed = newError(ProducerClosed, "producer already been closed") - errMemoryBufferIsFull = newError(ClientMemoryBufferIsFull, "client memory buffer is full") + 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") + ErrMetaTooLarge = newError(InvalidMessage, "message metadata size exceeds MaxMessageSize") + ErrProducerClosed = newError(ProducerClosed, "producer already been closed") + ErrMemoryBufferIsFull = newError(ClientMemoryBufferIsFull, "client memory buffer is full") + ErrSchema = newError(SchemaFailure, "schema error") + ErrTransaction = errors.New("transaction error") + ErrInvalidMessage = newError(InvalidMessage, "invalid message") + ErrTopicNotfound = newError(TopicNotFound, "topic not found") + ErrTopicTerminated = newError(TopicTerminated, "topic terminated") + ErrProducerBlockedQuotaExceeded = newError(ProducerBlockedQuotaExceededException, "producer blocked") + ErrProducerFenced = newError(ProducerFenced, "producer fenced") buffersPool sync.Pool sendRequestPool *sync.Pool @@ -449,25 +456,25 @@ func (p *partitionProducer) reconnectToBroker() { if strings.Contains(errMsg, errMsgTopicNotFound) { // when topic is deleted, we should give up reconnection. p.log.Warn("Topic not found, stop reconnecting, close the producer") - p.doClose(newError(TopicNotFound, err.Error())) + p.doClose(joinErrors(ErrTopicNotfound, err)) break } if strings.Contains(errMsg, errMsgTopicTerminated) { p.log.Warn("Topic was terminated, failing pending messages, stop reconnecting, close the producer") - p.doClose(newError(TopicTerminated, err.Error())) + p.doClose(joinErrors(ErrTopicTerminated, err)) break } if strings.Contains(errMsg, errMsgProducerBlockedQuotaExceededException) { p.log.Warn("Producer was blocked by quota exceed exception, failing pending messages, stop reconnecting") - p.failPendingMessages(newError(ProducerBlockedQuotaExceededException, err.Error())) + p.failPendingMessages(joinErrors(ErrProducerBlockedQuotaExceeded, err)) break } if strings.Contains(errMsg, errMsgProducerFenced) { p.log.Warn("Producer was fenced, failing pending messages, stop reconnecting") - p.doClose(newError(ProducerFenced, err.Error())) + p.doClose(joinErrors(ErrProducerFenced, err)) break } @@ -547,7 +554,7 @@ func (p *partitionProducer) internalSend(sr *sendRequest) { p.log.WithField("size", sr.uncompressedSize). WithField("properties", sr.msg.Properties). Error("unable to add message to batch") - sr.done(nil, errFailAddToBatch) + sr.done(nil, ErrFailAddToBatch) return } } @@ -802,7 +809,7 @@ func (p *partitionProducer) internalFlushCurrentBatch() { } if errors.Is(err, internal.ErrExceedMaxMessageSize) { - p.log.WithError(errMessageTooLarge).Errorf("internal err: %s", err) + p.log.WithError(ErrMessageTooLarge).Errorf("internal err: %s", err) } return @@ -893,11 +900,11 @@ func (p *partitionProducer) failTimeoutMessages() { for _, i := range pi.sendRequests { sr := i.(*sendRequest) - sr.done(nil, errSendTimeout) + sr.done(nil, ErrSendTimeout) } // flag the sending has completed with error, flush make no effect - pi.done(errSendTimeout) + pi.done(ErrSendTimeout) pi.Unlock() // finally reached the last view item, current iteration ends @@ -926,7 +933,7 @@ func (p *partitionProducer) internalFlushCurrentBatches() { } if errors.Is(errs[i], internal.ErrExceedMaxMessageSize) { - p.log.WithError(errMessageTooLarge).Errorf("internal err: %s", errs[i]) + p.log.WithError(ErrMessageTooLarge).Errorf("internal err: %s", errs[i]) return } @@ -1019,18 +1026,18 @@ func (p *partitionProducer) SendAsync(ctx context.Context, msg *ProducerMessage, func (p *partitionProducer) validateMsg(msg *ProducerMessage) error { if msg == nil { - return newError(InvalidMessage, "Message is nil") + return joinErrors(ErrInvalidMessage, fmt.Errorf("message is nil")) } if msg.Value != nil && msg.Payload != nil { - return newError(InvalidMessage, "Can not set Value and Payload both") + return joinErrors(ErrInvalidMessage, fmt.Errorf("can not set Value and Payload both")) } if p.options.DisableMultiSchema { if msg.Schema != nil && p.options.Schema != nil && msg.Schema.GetSchemaInfo().hash() != p.options.Schema.GetSchemaInfo().hash() { p.log.Errorf("The producer %s of the topic %s is disabled the `MultiSchema`", p.producerName, p.topic) - return fmt.Errorf("msg schema can not match with producer schema") + return joinErrors(ErrSchema, fmt.Errorf("msg schema can not match with producer schema")) } } @@ -1046,15 +1053,16 @@ func (p *partitionProducer) prepareTransaction(sr *sendRequest) error { if txn.state != TxnOpen { p.log.WithField("state", txn.state).Error("Failed to send message" + " by a non-open transaction.") - return newError(InvalidStatus, "Failed to send message by a non-open transaction.") + return joinErrors(ErrTransaction, + fmt.Errorf("failed to send message by a non-open transaction")) } if err := txn.registerProducerTopic(p.topic); err != nil { - return err + return joinErrors(ErrTransaction, err) } if err := txn.registerSendOrAckOp(); err != nil { - return err + return joinErrors(ErrTransaction, err) } sr.transaction = txn @@ -1080,7 +1088,7 @@ func (p *partitionProducer) updateSchema(sr *sendRequest) error { if schemaVersion == nil { schemaVersion, err = p.getOrCreateSchema(schema.GetSchemaInfo()) if err != nil { - return fmt.Errorf("get schema version fail, err: %w", err) + return joinErrors(ErrSchema, fmt.Errorf("get schema version fail, err: %w", err)) } p.schemaCache.Put(schema.GetSchemaInfo(), schemaVersion) } @@ -1097,7 +1105,7 @@ func (p *partitionProducer) updateUncompressedPayload(sr *sendRequest) error { if sr.msg.Value != nil { if sr.schema == nil { p.log.Errorf("Schema encode message failed %s", sr.msg.Value) - return newError(SchemaFailure, "set schema value without setting schema") + return joinErrors(ErrSchema, fmt.Errorf("set schema value without setting schema")) } // payload and schema are mutually exclusive @@ -1105,7 +1113,7 @@ func (p *partitionProducer) updateUncompressedPayload(sr *sendRequest) error { schemaPayload, err := sr.schema.Encode(sr.msg.Value) if err != nil { p.log.WithError(err).Errorf("Schema encode message failed %s", sr.msg.Value) - return newError(SchemaFailure, err.Error()) + return joinErrors(ErrSchema, err) } sr.uncompressedPayload = schemaPayload @@ -1160,11 +1168,11 @@ func (p *partitionProducer) updateChunkInfo(sr *sendRequest) error { // if msg is too large and chunking is disabled if checkSize > int64(sr.maxMessageSize) && !p.options.EnableChunking { - p.log.WithError(errMessageTooLarge). + p.log.WithError(ErrMessageTooLarge). WithField("size", checkSize). WithField("properties", sr.msg.Properties). Errorf("MaxMessageSize %d", sr.maxMessageSize) - return errMessageTooLarge + return ErrMessageTooLarge } if sr.sendAsBatch || !p.options.EnableChunking { @@ -1173,11 +1181,11 @@ func (p *partitionProducer) updateChunkInfo(sr *sendRequest) error { } else { sr.payloadChunkSize = int(sr.maxMessageSize) - proto.Size(sr.mm) if sr.payloadChunkSize <= 0 { - p.log.WithError(errMetaTooLarge). + p.log.WithError(ErrMetaTooLarge). WithField("metadata size", proto.Size(sr.mm)). WithField("properties", sr.msg.Properties). Errorf("MaxMessageSize %d", int(p._getConn().GetMaxMessageSize())) - return errMetaTooLarge + return ErrMetaTooLarge } // set ChunkMaxMessageSize if p.options.ChunkMaxMessageSize != 0 { @@ -1220,7 +1228,7 @@ func (p *partitionProducer) internalSendAsync( } if p.getProducerState() != producerReady { - sr.done(nil, errProducerClosed) + sr.done(nil, ErrProducerClosed) return } @@ -1333,7 +1341,7 @@ func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) func (p *partitionProducer) internalClose(req *closeProducer) { defer close(req.doneCh) - p.doClose(errProducerClosed) + p.doClose(ErrProducerClosed) } func (p *partitionProducer) doClose(reason error) { @@ -1508,11 +1516,11 @@ func (sr *sendRequest) done(msgID MessageID, err error) { WithField("properties", sr.msg.Properties) } - if errors.Is(err, errSendTimeout) { + if errors.Is(err, ErrSendTimeout) { sr.producer.metrics.PublishErrorsTimeout.Inc() } - if errors.Is(err, errMessageTooLarge) { + if errors.Is(err, ErrMessageTooLarge) { sr.producer.metrics.PublishErrorsMsgTooLarge.Inc() } @@ -1554,7 +1562,7 @@ func (p *partitionProducer) reserveSemaphore(sr *sendRequest) error { for i := 0; i < sr.totalChunks; i++ { if p.blockIfQueueFull() { if !p.publishSemaphore.Acquire(sr.ctx) { - return errContextExpired + return ErrContextExpired } // update sr.semaphore and sr.reservedSemaphore here so that we can release semaphore in the case @@ -1564,7 +1572,7 @@ func (p *partitionProducer) reserveSemaphore(sr *sendRequest) error { p.metrics.MessagesPending.Inc() } else { if !p.publishSemaphore.TryAcquire() { - return errSendQueueIsFull + return ErrSendQueueIsFull } // update sr.semaphore and sr.reservedSemaphore here so that we can release semaphore in the case @@ -1586,11 +1594,11 @@ func (p *partitionProducer) reserveMem(sr *sendRequest) error { if p.blockIfQueueFull() { if !p.client.memLimit.ReserveMemory(sr.ctx, requiredMem) { - return errContextExpired + return ErrContextExpired } } else { if !p.client.memLimit.TryReserveMemory(requiredMem) { - return errMemoryBufferIsFull + return ErrMemoryBufferIsFull } } diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 0f89069243..0d74cdeefe 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -1061,8 +1061,8 @@ func TestMaxMessageSize(t *testing.T) { assert.NoError(t, err) defer client.Close() - // Need to set BatchingMaxSize > serverMaxMessageSize to avoid errMessageTooLarge - // being masked by an earlier errFailAddToBatch + // Need to set BatchingMaxSize > serverMaxMessageSize to avoid ErrMessageTooLarge + // being masked by an earlier ErrFailAddToBatch producer, err := client.CreateProducer(ProducerOptions{ Topic: newTopicName(), BatchingMaxSize: uint(2 * serverMaxMessageSize), @@ -1088,7 +1088,7 @@ func TestMaxMessageSize(t *testing.T) { // So when bias <= 0, the uncompressed payload will not exceed maxMessageSize, // but encryptedPayloadSize exceeds maxMessageSize, Send() will return an internal error. // When bias = 1, the first check of maxMessageSize (for uncompressed payload) is valid, - // Send() will return errMessageTooLarge + // Send() will return ErrMessageTooLarge for bias := -1; bias <= 1; bias++ { payload := make([]byte, serverMaxMessageSize+bias) ID, err := producer.Send(context.Background(), &ProducerMessage{ @@ -1098,7 +1098,7 @@ func TestMaxMessageSize(t *testing.T) { assert.Equal(t, true, errors.Is(err, internal.ErrExceedMaxMessageSize)) assert.Nil(t, ID) } else { - assert.Equal(t, errMessageTooLarge, err) + assert.True(t, errors.Is(err, ErrMessageTooLarge)) } } @@ -1111,7 +1111,7 @@ func TestMaxMessageSize(t *testing.T) { assert.Equal(t, true, errors.Is(err, internal.ErrExceedMaxMessageSize)) assert.Nil(t, ID) } else { - assert.Equal(t, errMessageTooLarge, err) + assert.True(t, errors.Is(err, ErrMessageTooLarge)) } } } @@ -1190,8 +1190,7 @@ func TestTopicTermination(t *testing.T) { Payload: make([]byte, 1024), }) if err != nil { - e := err.(*Error) - if e.result == TopicTerminated || err == errProducerClosed { + if errors.Is(err, ErrTopicTerminated) || errors.Is(err, ErrProducerClosed) { terminatedChan <- true } else { terminatedChan <- false @@ -2348,7 +2347,7 @@ func TestFailPendingMessageWithClose(t *testing.T) { Payload: make([]byte, 1024), }, func(id MessageID, message *ProducerMessage, e error) { if e != nil { - assert.Equal(t, errProducerClosed, e) + assert.True(t, errors.Is(e, ErrProducerClosed)) } }) } From 3a1a766e9cc9e9c3afa78ea9e078aba9131ad2da Mon Sep 17 00:00:00 2001 From: zhou zhuohan <843520313@qq.com> Date: Tue, 2 Jan 2024 14:45:43 +0800 Subject: [PATCH 307/348] docs: update Unsubscribe interface comment (#1146) Co-authored-by: ninjazhou --- pulsar/consumer.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pulsar/consumer.go b/pulsar/consumer.go index a62eabe1ed..667bff66cd 100644 --- a/pulsar/consumer.go +++ b/pulsar/consumer.go @@ -254,6 +254,12 @@ type Consumer interface { Subscription() string // Unsubscribe the consumer + // + // Unsubscribing will cause the subscription to be deleted, + // and all the retained data can potentially be deleted based on message retention and ttl policy. + // + // This operation will fail when performed on a shared subscription + // where more than one consumer are currently connected. Unsubscribe() error // Receive a single message. From cae74c52064df302633acfa4fc9ad073cf2848ae Mon Sep 17 00:00:00 2001 From: Tom Jorissen Date: Tue, 9 Jan 2024 05:00:10 +0100 Subject: [PATCH 308/348] [Issue 1105][pulsaradmin] fix AutoTopicCreation for type non-partitioned (#1107) Fixes #1105 ### Motivation To allow setting the AutoTopicCreation policy to non-partitioned it is required to not send the partitions parameter, by making Partitions a pointer instead of an int this is possible. ### Modifications Change the type of Partitions in TopicAutoCreationConfig from int to *int --- pulsaradmin/pkg/utils/topic_auto_creation_config.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/utils/topic_auto_creation_config.go b/pulsaradmin/pkg/utils/topic_auto_creation_config.go index 6664655974..8444514250 100644 --- a/pulsaradmin/pkg/utils/topic_auto_creation_config.go +++ b/pulsaradmin/pkg/utils/topic_auto_creation_config.go @@ -20,5 +20,5 @@ package utils type TopicAutoCreationConfig struct { Allow bool `json:"allowAutoTopicCreation"` Type TopicType `json:"topicType"` - Partitions int `json:"defaultNumPartitions"` + Partitions *int `json:"defaultNumPartitions"` } From ad7887e3f42171681a6fd4b16d7b4b8d389f5f89 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Tue, 9 Jan 2024 12:00:36 +0800 Subject: [PATCH 309/348] Add test for admin topic creation (#1152) ### Motivation Lacking tests for creating topic using pulsarAdmin ### Modifications - Add test for admin topic creation --- pulsaradmin/pkg/admin/topic_test.go | 55 +++++++++++++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 pulsaradmin/pkg/admin/topic_test.go diff --git a/pulsaradmin/pkg/admin/topic_test.go b/pulsaradmin/pkg/admin/topic_test.go new file mode 100644 index 0000000000..06c33f2ef5 --- /dev/null +++ b/pulsaradmin/pkg/admin/topic_test.go @@ -0,0 +1,55 @@ +// 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 admin + +import ( + "testing" + + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" +) + +func TestCreateTopic(t *testing.T) { + checkError := func(err error) { + if err != nil { + t.Error(err) + } + } + + cfg := &config.Config{} + admin, err := New(cfg) + checkError(err) + + topic := "persistent://public/default/testCreateTopic" + + topicName, err := utils.GetTopicName(topic) + checkError(err) + + err = admin.Topics().Create(*topicName, 0) + checkError(err) + + topicLists, err := admin.Namespaces().GetTopics("public/default") + checkError(err) + + for _, t := range topicLists { + if t == topic { + return + } + } + t.Error("Couldn't find topic: " + topic) +} From 3388eae9f03f43b1446c3dd573a104b3f95fe302 Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Tue, 9 Jan 2024 12:01:50 +0800 Subject: [PATCH 310/348] [Improve] Implement GetTopicAutoCreation (#1151) ### Motivation The `GetTopicAutoCreation` endpoint is missed, needs to add it ### Modifications Implement the `GetTopicAutoCreation` method --- pulsaradmin/pkg/admin/namespace.go | 10 ++ pulsaradmin/pkg/admin/namespace_test.go | 175 ++++++++++++++++++++++++ 2 files changed, 185 insertions(+) create mode 100644 pulsaradmin/pkg/admin/namespace_test.go diff --git a/pulsaradmin/pkg/admin/namespace.go b/pulsaradmin/pkg/admin/namespace.go index 732441e8c2..782ae3ae25 100644 --- a/pulsaradmin/pkg/admin/namespace.go +++ b/pulsaradmin/pkg/admin/namespace.go @@ -75,6 +75,9 @@ type Namespaces interface { // RemoveBacklogQuota removes a backlog quota policy from a namespace RemoveBacklogQuota(namespace string) error + // GetTopicAutoCreation returns the topic auto-creation config for a namespace + GetTopicAutoCreation(namespace utils.NameSpaceName) (*utils.TopicAutoCreationConfig, error) + // SetTopicAutoCreation sets topic auto-creation config for a namespace, overriding broker settings SetTopicAutoCreation(namespace utils.NameSpaceName, config utils.TopicAutoCreationConfig) error @@ -445,6 +448,13 @@ func (n *namespaces) RemoveBacklogQuota(namespace string) error { return n.pulsar.Client.DeleteWithQueryParams(endpoint, params) } +func (n *namespaces) GetTopicAutoCreation(namespace utils.NameSpaceName) (*utils.TopicAutoCreationConfig, error) { + var topicAutoCreation utils.TopicAutoCreationConfig + endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "autoTopicCreation") + err := n.pulsar.Client.Get(endpoint, &topicAutoCreation) + return &topicAutoCreation, err +} + func (n *namespaces) SetTopicAutoCreation(namespace utils.NameSpaceName, config utils.TopicAutoCreationConfig) error { endpoint := n.pulsar.endpoint(n.basePath, namespace.String(), "autoTopicCreation") return n.pulsar.Client.Post(endpoint, &config) diff --git a/pulsaradmin/pkg/admin/namespace_test.go b/pulsaradmin/pkg/admin/namespace_test.go new file mode 100644 index 0000000000..f934a96865 --- /dev/null +++ b/pulsaradmin/pkg/admin/namespace_test.go @@ -0,0 +1,175 @@ +// 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 admin + +import ( + "testing" + + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/rest" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func ptr(n int) *int { + return &n +} + +func TestSetTopicAutoCreation(t *testing.T) { + config := &config.Config{} + admin, err := New(config) + require.NoError(t, err) + require.NotNil(t, admin) + + tests := []struct { + name string + namespace string + config utils.TopicAutoCreationConfig + errReason string + }{ + { + name: "Set partitioned type topic auto creation", + namespace: "public/default", + config: utils.TopicAutoCreationConfig{ + Allow: true, + Type: utils.Partitioned, + Partitions: ptr(3), + }, + errReason: "", + }, + { + name: "Set partitioned type topic auto creation without partitions", + namespace: "public/default", + config: utils.TopicAutoCreationConfig{ + Allow: true, + Type: utils.Partitioned, + }, + errReason: "Invalid configuration for autoTopicCreationOverride. the detail is [defaultNumPartitions] " + + "cannot be null when the type is partitioned.", + }, + { + name: "Set partitioned type topic auto creation with partitions < 1", + namespace: "public/default", + config: utils.TopicAutoCreationConfig{ + Allow: true, + Type: utils.Partitioned, + Partitions: ptr(-1), + }, + errReason: "Invalid configuration for autoTopicCreationOverride. the detail is [defaultNumPartitions] " + + "cannot be less than 1 for partition type.", + }, + { + name: "Set non-partitioned type topic auto creation", + namespace: "public/default", + config: utils.TopicAutoCreationConfig{ + Allow: true, + Type: utils.NonPartitioned, + }, + errReason: "", + }, + { + name: "Set non-partitioned type topic auto creation with partitions", + namespace: "public/default", + config: utils.TopicAutoCreationConfig{ + Allow: true, + Type: utils.NonPartitioned, + Partitions: ptr(3), + }, + errReason: "Invalid configuration for autoTopicCreationOverride. the detail is [defaultNumPartitions] is " + + "not allowed to be set when the type is non-partition.", + }, + { + name: "Disable topic auto creation", + namespace: "public/default", + config: utils.TopicAutoCreationConfig{ + Allow: false, + }, + errReason: "", + }, + { + name: "Set topic auto creation on a non-exist namespace", + namespace: "public/nonexist", + config: utils.TopicAutoCreationConfig{ + Allow: true, + Type: utils.NonPartitioned, + }, + errReason: "Namespace does not exist", + }, + { + name: "Set topic auto creation on a non-exist tenant", + namespace: "non-exist/default", + config: utils.TopicAutoCreationConfig{ + Allow: true, + Type: utils.NonPartitioned, + }, + errReason: "Tenant does not exist", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + namespace, _ := utils.GetNamespaceName(tt.namespace) + err := admin.Namespaces().SetTopicAutoCreation(*namespace, tt.config) + if tt.errReason == "" { + assert.Equal(t, nil, err) + + err = admin.Namespaces().RemoveTopicAutoCreation(*namespace) + assert.Equal(t, nil, err) + } + if err != nil { + restError := err.(rest.Error) + assert.Equal(t, tt.errReason, restError.Reason) + } + }) + } +} + +func TestGetTopicAutoCreation(t *testing.T) { + config := &config.Config{} + admin, err := New(config) + require.NoError(t, err) + require.NotNil(t, admin) + + namespace, _ := utils.GetNamespaceName("public/default") + + // set the topic auto creation config and get it + err = admin.Namespaces().SetTopicAutoCreation(*namespace, utils.TopicAutoCreationConfig{ + Allow: true, + Type: utils.NonPartitioned, + }) + assert.Equal(t, nil, err) + topicAutoCreation, err := admin.Namespaces().GetTopicAutoCreation(*namespace) + assert.Equal(t, nil, err) + expected := utils.TopicAutoCreationConfig{ + Allow: true, + Type: utils.NonPartitioned, + } + assert.Equal(t, expected, *topicAutoCreation) + + // remove the topic auto creation config and get it + err = admin.Namespaces().RemoveTopicAutoCreation(*namespace) + assert.Equal(t, nil, err) + + topicAutoCreation, err = admin.Namespaces().GetTopicAutoCreation(*namespace) + assert.Equal(t, nil, err) + expected = utils.TopicAutoCreationConfig{ + Allow: false, + Type: "", + } + assert.Equal(t, expected, *topicAutoCreation) +} From 58941348a7fa467df21ddd9a3ddf11e9103cc4ac Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 9 Jan 2024 19:01:14 -0800 Subject: [PATCH 311/348] chore(deps): bump github.com/dvsekhvalnov/jose2go from 1.5.0 to 1.6.0 (#1150) Bumps [github.com/dvsekhvalnov/jose2go](/~https://github.com/dvsekhvalnov/jose2go) from 1.5.0 to 1.6.0. - [Commits](/~https://github.com/dvsekhvalnov/jose2go/compare/v1.5...v1.6.0) --- updated-dependencies: - dependency-name: github.com/dvsekhvalnov/jose2go dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 8 +++++--- go.sum | 4 ++-- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 52bfe43a96..df1024759e 100644 --- a/go.mod +++ b/go.mod @@ -30,7 +30,10 @@ require ( google.golang.org/protobuf v1.30.0 ) -require github.com/golang/protobuf v1.5.2 +require ( + github.com/golang/protobuf v1.5.2 + github.com/hashicorp/go-multierror v1.1.1 +) require ( github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect @@ -38,13 +41,12 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.1.1 // indirect github.com/danieljoos/wincred v1.1.2 // indirect - github.com/dvsekhvalnov/jose2go v1.5.0 // indirect + github.com/dvsekhvalnov/jose2go v1.6.0 // indirect github.com/fsnotify/fsnotify v1.4.9 // indirect github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect github.com/golang/snappy v0.0.1 // indirect github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/hashicorp/errwrap v1.0.0 // indirect - github.com/hashicorp/go-multierror v1.1.1 // indirect github.com/inconshreveable/mousetrap v1.0.1 // indirect github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect diff --git a/go.sum b/go.sum index e8a9e76be7..b75adeae6e 100644 --- a/go.sum +++ b/go.sum @@ -74,8 +74,8 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dimfeld/httptreemux v5.0.1+incompatible h1:Qj3gVcDNoOthBAqftuD596rm4wg/adLLz5xh5CmpiCA= github.com/dimfeld/httptreemux v5.0.1+incompatible/go.mod h1:rbUlSV+CCpv/SuqUTP/8Bk2O3LyUV436/yaRGkhP6Z0= -github.com/dvsekhvalnov/jose2go v1.5.0 h1:3j8ya4Z4kMCwT5nXIKFSV84YS+HdqSSO0VsTQxaLAeM= -github.com/dvsekhvalnov/jose2go v1.5.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= +github.com/dvsekhvalnov/jose2go v1.6.0 h1:Y9gnSnP4qEI0+/uQkHvFXeD2PLPJeXEL+ySMEA2EjTY= +github.com/dvsekhvalnov/jose2go v1.6.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= From 93476909e36b7344c1b623fbe1b0d5b93792eb2f Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 12 Jan 2024 16:03:55 +0800 Subject: [PATCH 312/348] chore(deps): bump golang.org/x/net from 0.0.0-20220225172249-27dd8689420f to 0.17.0 (#1155) Golang 1.16 is nolonger compatible with pulsar-client-go because `unsafe.Slice` is not defined. Update the CI workflows to test the latest 4 Golang major releases. Actually Golang only maintains the latest two major releases, see https://go.dev/doc/devel/release#policy --- .github/workflows/ci.yml | 2 +- go.mod | 10 +++++----- go.sum | 10 ++++++++++ 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index f0c9b63999..9ad0b3f255 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -38,7 +38,7 @@ jobs: runs-on: ubuntu-latest strategy: matrix: - go-version: [1.16, 1.17, 1.18, 1.19] + go-version: ['1.18', '1.19', '1.20', '1.21'] steps: - uses: actions/checkout@v3 - name: clean docker cache diff --git a/go.mod b/go.mod index df1024759e..3257d51047 100644 --- a/go.mod +++ b/go.mod @@ -24,7 +24,7 @@ require ( github.com/spf13/cobra v1.6.1 github.com/stretchr/testify v1.8.0 go.uber.org/atomic v1.7.0 - golang.org/x/mod v0.5.1 + golang.org/x/mod v0.8.0 golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 golang.org/x/time v0.0.0-20191024005414-555d28b269f0 google.golang.org/protobuf v1.30.0 @@ -58,10 +58,10 @@ require ( github.com/prometheus/procfs v0.6.0 // indirect github.com/spf13/pflag v1.0.5 // indirect github.com/stretchr/objx v0.4.0 // indirect - golang.org/x/net v0.0.0-20220225172249-27dd8689420f // indirect - golang.org/x/sys v0.0.0-20220204135822-1c1b9b1eba6a // indirect - golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 // indirect - golang.org/x/text v0.3.7 // indirect + golang.org/x/net v0.17.0 // indirect + golang.org/x/sys v0.13.0 // indirect + golang.org/x/term v0.13.0 // indirect + golang.org/x/text v0.13.0 // indirect google.golang.org/appengine v1.6.7 // indirect gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect diff --git a/go.sum b/go.sum index b75adeae6e..50a1ba3ebb 100644 --- a/go.sum +++ b/go.sum @@ -323,6 +323,8 @@ golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.5.1 h1:OJxoQ/rynoF0dcCdI7cLPktw/hR2cueqYfjm43oqK38= golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= +golang.org/x/mod v0.8.0 h1:LUYupSeNrTNCGzR/hVBk2NHZO4hXcVaW1k4Qx7rjPx8= +golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -357,6 +359,8 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220225172249-27dd8689420f h1:oA4XRj0qtSt8Yo1Zms0CUlsT3KG69V2UGQWPBxujDmc= golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= +golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -418,9 +422,13 @@ golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210819135213-f52c844e1c1c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220204135822-1c1b9b1eba6a h1:ppl5mZgokTT8uPkmYOyEUmPTr3ypaKkg5eFOGrAmxxE= golang.org/x/sys v0.0.0-20220204135822-1c1b9b1eba6a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= +golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 h1:JGgROgKl9N8DuW20oFS5gxc+lE67/N3FcwmBPMe7ArY= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.13.0 h1:bb+I9cTfFazGW51MZqBVmZy7+JEJMouUHTUSKVQLBek= +golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -429,6 +437,8 @@ golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.13.0 h1:ablQoSUd0tRdKxZewP80B+BaqeKJuVhuRxj/dkrun3k= +golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0 h1:/5xXl8Y5W96D+TtHSlonuFqGHIWVuyCkGJLwGh9JJFs= From 4e138228fe501ec39856afbc5e541e87a143b73f Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Mon, 15 Jan 2024 10:02:23 +0800 Subject: [PATCH 313/348] [fix] Fix DLQ producer name conflicts when multiples consumers send messages to DLQ (#1156) ### Motivation To keep consistent with the Java client. Releted PR: /~https://github.com/apache/pulsar/pull/21890 ### Modifications Set DLQ producerName `fmt.Sprintf("%s-%s-%s-DLQ", r.topicName, r.subscriptionName, r.consumerName)` --- pulsar/consumer_impl.go | 2 +- pulsar/consumer_regex_test.go | 6 ++++-- pulsar/consumer_test.go | 6 ++++-- pulsar/dlq_router.go | 6 ++++-- pulsar/reader_impl.go | 2 +- 5 files changed, 14 insertions(+), 8 deletions(-) diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index d701ab16d6..75d839b412 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -167,7 +167,7 @@ func newConsumer(client *client, options ConsumerOptions) (Consumer, error) { } } - dlq, err := newDlqRouter(client, options.DLQ, options.Topic, options.SubscriptionName, client.log) + dlq, err := newDlqRouter(client, options.DLQ, options.Topic, options.SubscriptionName, options.Name, client.log) if err != nil { return nil, err } diff --git a/pulsar/consumer_regex_test.go b/pulsar/consumer_regex_test.go index 3e5f1d61db..ebd7e4e196 100644 --- a/pulsar/consumer_regex_test.go +++ b/pulsar/consumer_regex_test.go @@ -152,9 +152,10 @@ func runRegexConsumerDiscoverPatternAll(t *testing.T, c Client, namespace string opts := ConsumerOptions{ SubscriptionName: "regex-sub", AutoDiscoveryPeriod: 5 * time.Minute, + Name: "regex-consumer", } - dlq, _ := newDlqRouter(c.(*client), nil, tn.Topic, "regex-sub", log.DefaultNopLogger()) + dlq, _ := newDlqRouter(c.(*client), nil, tn.Topic, "regex-sub", "regex-consumer", log.DefaultNopLogger()) rlq, _ := newRetryRouter(c.(*client), nil, false, log.DefaultNopLogger()) consumer, err := newRegexConsumer(c.(*client), opts, tn, pattern, make(chan ConsumerMessage, 1), dlq, rlq) if err != nil { @@ -190,9 +191,10 @@ func runRegexConsumerDiscoverPatternFoo(t *testing.T, c Client, namespace string opts := ConsumerOptions{ SubscriptionName: "regex-sub", AutoDiscoveryPeriod: 5 * time.Minute, + Name: "regex-consumer", } - dlq, _ := newDlqRouter(c.(*client), nil, tn.Topic, "regex-sub", log.DefaultNopLogger()) + dlq, _ := newDlqRouter(c.(*client), nil, tn.Topic, "regex-sub", "regex-consumer", log.DefaultNopLogger()) rlq, _ := newRetryRouter(c.(*client), nil, false, log.DefaultNopLogger()) consumer, err := newRegexConsumer(c.(*client), opts, tn, pattern, make(chan ConsumerMessage, 1), dlq, rlq) if err != nil { diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 8b983d0d19..df70b0dd0b 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -1449,13 +1449,15 @@ func DLQWithProducerOptions(t *testing.T, prodOpt *ProducerOptions) { if prodOpt != nil { dlqPolicy.ProducerOptions = *prodOpt } - sub := "my-sub" + sub, consumerName := "my-sub", "my-consumer" + consumer, err := client.Subscribe(ConsumerOptions{ Topic: topic, SubscriptionName: sub, NackRedeliveryDelay: 1 * time.Second, Type: Shared, DLQ: &dlqPolicy, + Name: consumerName, }) assert.Nil(t, err) defer consumer.Close() @@ -1508,7 +1510,7 @@ func DLQWithProducerOptions(t *testing.T, prodOpt *ProducerOptions) { assert.Equal(t, []byte(expectMsg), msg.Payload()) // check dql produceName - assert.Equal(t, msg.ProducerName(), fmt.Sprintf("%s-%s-DLQ", topic, sub)) + assert.Equal(t, msg.ProducerName(), fmt.Sprintf("%s-%s-%s-DLQ", topic, sub, consumerName)) // check original messageId assert.NotEmpty(t, msg.Properties()[PropertyOriginMessageID]) diff --git a/pulsar/dlq_router.go b/pulsar/dlq_router.go index 5b9314bddc..6be35d7485 100644 --- a/pulsar/dlq_router.go +++ b/pulsar/dlq_router.go @@ -34,16 +34,18 @@ type dlqRouter struct { closeCh chan interface{} topicName string subscriptionName string + consumerName string log log.Logger } -func newDlqRouter(client Client, policy *DLQPolicy, topicName, subscriptionName string, +func newDlqRouter(client Client, policy *DLQPolicy, topicName, subscriptionName, consumerName string, logger log.Logger) (*dlqRouter, error) { r := &dlqRouter{ client: client, policy: policy, topicName: topicName, subscriptionName: subscriptionName, + consumerName: consumerName, log: logger, } @@ -159,7 +161,7 @@ func (r *dlqRouter) getProducer(schema Schema) Producer { opt.Topic = r.policy.DeadLetterTopic opt.Schema = schema if opt.Name == "" { - opt.Name = fmt.Sprintf("%s-%s-DLQ", r.topicName, r.subscriptionName) + opt.Name = fmt.Sprintf("%s-%s-%s-DLQ", r.topicName, r.subscriptionName, r.consumerName) } // the origin code sets to LZ4 compression with no options diff --git a/pulsar/reader_impl.go b/pulsar/reader_impl.go index 0999e88fee..7b260b88db 100644 --- a/pulsar/reader_impl.go +++ b/pulsar/reader_impl.go @@ -127,7 +127,7 @@ func newReader(client *client, options ReaderOptions) (Reader, error) { } // Provide dummy dlq router with not dlq policy - dlq, err := newDlqRouter(client, nil, options.Topic, options.SubscriptionName, client.log) + dlq, err := newDlqRouter(client, nil, options.Topic, options.SubscriptionName, options.Name, client.log) if err != nil { return nil, err } From 5768f009c3a28cf58453acf7f67dcdeb6c7ff5d3 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Mon, 29 Jan 2024 09:39:10 +0800 Subject: [PATCH 314/348] Add 0.12.0 change log (#1153) * Add 0.12.0 change log --- CHANGELOG.md | 79 ++++++++++++++++++++++++++++++++++++++++++++++++++++ VERSION | 2 +- stable.txt | 2 +- 3 files changed, 81 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d7afe2c3a0..7a4c81cb74 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,6 +23,85 @@ All notable changes to this project will be documented in this file. +[0.12.0] 2024-01-10 + +## What's Changed +* Improved the performance of schema and schema cache by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1033 +* Fixed return when registerSendOrAckOp() failed by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1045 +* Fixed the incorrect link in the release process by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/1050 +* Fixed Producer by checking if message is nil by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1047 +* Added 0.11.0 change log by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/1048 +* Fixed 0.11.0 change log by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/1054 +* Fixed issue 877 where ctx in partitionProducer.Send() was not performing as expected by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/1053 +* Fixed Producer by stopping block request even if Value and Payload are both set by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1052 +* Improved Producer by simplifying the flush logic by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1049 +* Fixed issue 1051: inaccurate producer memory limit in chunking and schema by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/1055 +* Fixed issue by sending Close Command on Producer/Consumer create timeout by @michaeljmarshall in /~https://github.com/apache/pulsar-client-go/pull/1061 +* Fixed issue 1057: producer flush operation is not guaranteed to flush all messages by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/1058 +* Fixed issue 1064: panic when trying to flush in DisableBatching=true by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/1065 +* Fixed transaction acknowledgement and send logic for chunk message by @liangyepianzhou in /~https://github.com/apache/pulsar-client-go/pull/1069 +* Fixed issue by closing consumer resources if creation fails by @michaeljmarshall in /~https://github.com/apache/pulsar-client-go/pull/1070 +* Fixed issue where client reconnected every authenticationRefreshCheckSeconds when using TLS authentication by @jffp113 in /~https://github.com/apache/pulsar-client-go/pull/1062 +* Corrected the SendAsync() description by @Gleiphir2769 in /~https://github.com/apache/pulsar-client-go/pull/1066 +* CI: replaced license header checker and formatter by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/1077 +* Chore: allowed rebase and merge by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/1080 +* Adopted pulsar-admin-go sources by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/1079 +* Reverted: allowed rebase and merge by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/1081 +* Fixed producer by failing all messages that are pending requests when closing like Java by @graysonzeng in /~https://github.com/apache/pulsar-client-go/pull/1059 +* Supported load config from env by @tuteng in /~https://github.com/apache/pulsar-client-go/pull/1089 +* Fixed issue where multiple calls to client.Close causes panic by @crossoverJie in /~https://github.com/apache/pulsar-client-go/pull/1046 +* Improved client by implementing GetLastMSgID for Reader by @liangyepianzhou in /~https://github.com/apache/pulsar-client-go/pull/1087 +* Fixed comment for ConnectionMaxIdleTime by @massakam in /~https://github.com/apache/pulsar-client-go/pull/1091 +* Issue 1094: connectionTimeout respects net.Dialer default timeout by @zzzming in /~https://github.com/apache/pulsar-client-go/pull/1095 +* Supported OAuth2 with scope field by @labuladong in /~https://github.com/apache/pulsar-client-go/pull/1097 +* Fixed issue where DisableReplication flag does not work by @massakam in /~https://github.com/apache/pulsar-client-go/pull/1100 +* Double-checked before consumer reconnect by @zccold in /~https://github.com/apache/pulsar-client-go/pull/1084 +* Fixed schema error by @leizhiyuan in /~https://github.com/apache/pulsar-client-go/pull/823 +* PR-1071-1: renamed pendingItem.Complete() to pendingItem.done() by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1109 +* PR-1071-2: added sendRequest.done() to release resource together by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1110 +* Refactor: factored out validateMsg by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/1117 +* Refactor: factored out prepareTransaction by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/1118 +* Completed comment on ProducerInterceptor interface BeforeSend method by @ojcm in /~https://github.com/apache/pulsar-client-go/pull/1119 +* Refactor: prepared sendrequest and moved to internalSendAsync by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/1120 +* Fix: normalized all send request resource release into sr.done by @tisonkun in /~https://github.com/apache/pulsar-client-go/pull/1121 +* Improvement: added func blockIfQueueFull() to encapsulate DisableBlockIfQue… by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1122 +* Improved debug log clarity in ReceivedSendReceipt() by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1123 +* Fixed issue 1098 by checking batchBuilder in case batch is disabled by @zzzming in /~https://github.com/apache/pulsar-client-go/pull/1099 +* Fixed Producer by fixing reconnection backoff logic by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1125 +* Added 0.11.1 change log by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/1092 +* Fixed dead link to the KEYS file in the release process by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/1127 +* Improved performance by pooling sendRequest by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1126 +* Fixed argument order to Errorf in TableView message handling by @ojcm in /~https://github.com/apache/pulsar-client-go/pull/1130 +* Fixed Producer by double-checking before reconnect by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1131 +* Fixed issue where client must not retry connecting to broker when topic is terminated by @pkumar-singh in /~https://github.com/apache/pulsar-client-go/pull/1128 +* Issue 1132: Fixed JSONSchema unmarshalling in TableView by @ojcm in /~https://github.com/apache/pulsar-client-go/pull/1133 +* Improved by setting dlq producerName by @crossoverJie in /~https://github.com/apache/pulsar-client-go/pull/1137 +* Fixed channel deadlock in regexp consumer by @goncalo-rodrigues in /~https://github.com/apache/pulsar-client-go/pull/1141 +* Fixed Producer: handled TopicNotFound/TopicTerminated/ProducerBlockedQuotaExceededException/ProducerFenced when reconnecting by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1134 +* Transaction: Avoided a panic when using transaction by @Gilthoniel in /~https://github.com/apache/pulsar-client-go/pull/1144 +* Improved by updating connection.lastDataReceivedTime when connection is ready by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1145 +* Improved Producer by normalizing and exporting the errors by @gunli in /~https://github.com/apache/pulsar-client-go/pull/1143 +* Updated Unsubscribe() interface comment by @geniusjoe in /~https://github.com/apache/pulsar-client-go/pull/1146 +* Issue 1105: Fixed AutoTopicCreation for type non-partitioned by @tomjo in /~https://github.com/apache/pulsar-client-go/pull/1107 +* Added test for admin topic creation by @RobertIndie in /~https://github.com/apache/pulsar-client-go/pull/1152 +* Implemented GetTopicAutoCreation by @jiangpengcheng in /~https://github.com/apache/pulsar-client-go/pull/1151 +* Bumped github.com/dvsekhvalnov/jose2go from 1.5.0 to 1.6.0 by @dependabot in /~https://github.com/apache/pulsar-client-go/pull/1150 +* Bump golang.org/x/net from 0.0.0-20220225172249-27dd8689420f to 0.17.0 by @BewareMyPower in /~https://github.com/apache/pulsar-client-go/pull/1155 +* Fix DLQ producer name conflicts when multiples consumers send messages to DLQ by @crossoverJie in /~https://github.com/apache/pulsar-client-go/pull/1156 + +## New Contributors +* @jffp113 made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1062 +* @tuteng made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1089 +* @zccold made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1084 +* @ojcm made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1119 +* @pkumar-singh made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1128 +* @goncalo-rodrigues made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1141 +* @Gilthoniel made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1144 +* @geniusjoe made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1146 +* @tomjo made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1107 +* @jiangpengcheng made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1151 +* @dependabot made their first contribution in /~https://github.com/apache/pulsar-client-go/pull/1150 + [0.11.1] 2023-09-11 - Close consumer resources if the creation fails by @michaeljmarshall in [#1070](/~https://github.com/apache/pulsar-client-go/pull/1070) diff --git a/VERSION b/VERSION index dbf0637bab..725659e7c0 100644 --- a/VERSION +++ b/VERSION @@ -1,3 +1,3 @@ // This version number refers to the currently released version number // Please fix the version when release. -v0.11.1 +v0.12.0 diff --git a/stable.txt b/stable.txt index 7e38472ca7..8a5f2e8dfe 100644 --- a/stable.txt +++ b/stable.txt @@ -1,3 +1,3 @@ // This version number refers to the current stable version, generally is `VERSION - 1`. // Please fix the version when release. -v0.11.1 +v0.12.0 \ No newline at end of file From b0487429672a1f9939335f47fd48551625c3eb54 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 31 Jan 2024 16:37:20 +0800 Subject: [PATCH 315/348] Fix SIGSEGV with zstd compression enabled --- pulsar/internal/compression/zstd_cgo.go | 5 +++++ pulsar/producer_test.go | 28 +++++++++++++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/pulsar/internal/compression/zstd_cgo.go b/pulsar/internal/compression/zstd_cgo.go index 25429e2512..3b32323001 100644 --- a/pulsar/internal/compression/zstd_cgo.go +++ b/pulsar/internal/compression/zstd_cgo.go @@ -25,6 +25,8 @@ package compression import ( + "sync" + "github.com/DataDog/zstd" log "github.com/sirupsen/logrus" ) @@ -33,6 +35,7 @@ type zstdCGoProvider struct { ctx zstd.Ctx level Level zstdLevel int + mu sync.Mutex } func newCGoZStdProvider(level Level) Provider { @@ -61,6 +64,8 @@ func (z *zstdCGoProvider) CompressMaxSize(originalSize int) int { } func (z *zstdCGoProvider) Compress(dst, src []byte) []byte { + z.mu.Lock() + defer z.mu.Unlock() out, err := z.ctx.CompressLevel(dst, src, z.zstdLevel) if err != nil { log.WithError(err).Fatal("Failed to compress") diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 0d74cdeefe..3b9ea7e8da 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -2357,6 +2357,34 @@ func TestFailPendingMessageWithClose(t *testing.T) { assert.Equal(t, 0, partitionProducerImp.pendingQueue.Size()) } +func TestSendConcurrently(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + assert.NoError(t, err) + defer client.Close() + testProducer, err := client.CreateProducer(ProducerOptions{ + Topic: newTopicName(), + CompressionType: ZSTD, + CompressionLevel: Better, + DisableBatching: true, + }) + assert.NoError(t, err) + + var wg sync.WaitGroup + for i := 0; i < 100; i++ { + wg.Add(1) + go func() { + _, err := testProducer.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 100), + }) + assert.NoError(t, err) + wg.Done() + }() + } + wg.Wait() +} + type pendingQueueWrapper struct { pendingQueue internal.BlockingQueue writtenBuffers *[]internal.Buffer From 1ebc162a98156972ccddd5300b5dbffe15e901c1 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 31 Jan 2024 16:44:26 +0800 Subject: [PATCH 316/348] Revert "Fix SIGSEGV with zstd compression enabled" This reverts commit b0487429672a1f9939335f47fd48551625c3eb54. --- pulsar/internal/compression/zstd_cgo.go | 5 ----- pulsar/producer_test.go | 28 ------------------------- 2 files changed, 33 deletions(-) diff --git a/pulsar/internal/compression/zstd_cgo.go b/pulsar/internal/compression/zstd_cgo.go index 3b32323001..25429e2512 100644 --- a/pulsar/internal/compression/zstd_cgo.go +++ b/pulsar/internal/compression/zstd_cgo.go @@ -25,8 +25,6 @@ package compression import ( - "sync" - "github.com/DataDog/zstd" log "github.com/sirupsen/logrus" ) @@ -35,7 +33,6 @@ type zstdCGoProvider struct { ctx zstd.Ctx level Level zstdLevel int - mu sync.Mutex } func newCGoZStdProvider(level Level) Provider { @@ -64,8 +61,6 @@ func (z *zstdCGoProvider) CompressMaxSize(originalSize int) int { } func (z *zstdCGoProvider) Compress(dst, src []byte) []byte { - z.mu.Lock() - defer z.mu.Unlock() out, err := z.ctx.CompressLevel(dst, src, z.zstdLevel) if err != nil { log.WithError(err).Fatal("Failed to compress") diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 3b9ea7e8da..0d74cdeefe 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -2357,34 +2357,6 @@ func TestFailPendingMessageWithClose(t *testing.T) { assert.Equal(t, 0, partitionProducerImp.pendingQueue.Size()) } -func TestSendConcurrently(t *testing.T) { - client, err := NewClient(ClientOptions{ - URL: lookupURL, - }) - assert.NoError(t, err) - defer client.Close() - testProducer, err := client.CreateProducer(ProducerOptions{ - Topic: newTopicName(), - CompressionType: ZSTD, - CompressionLevel: Better, - DisableBatching: true, - }) - assert.NoError(t, err) - - var wg sync.WaitGroup - for i := 0; i < 100; i++ { - wg.Add(1) - go func() { - _, err := testProducer.Send(context.Background(), &ProducerMessage{ - Payload: make([]byte, 100), - }) - assert.NoError(t, err) - wg.Done() - }() - } - wg.Wait() -} - type pendingQueueWrapper struct { pendingQueue internal.BlockingQueue writtenBuffers *[]internal.Buffer From 877613503b70c4ee67a8408f31881d88fc086456 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Fri, 2 Feb 2024 01:24:50 +0800 Subject: [PATCH 317/348] Fix SIGSEGV with zstd compression enabled (#1164) * Fix SIGSEGV with zstd compression enabled * Use sync.Pool to cache zstd ctx * Fix race in sequenceID assignment * Fix GetAndAdd --- pulsar/internal/compression/zstd_cgo.go | 16 ++++++++++---- pulsar/internal/utils.go | 2 +- pulsar/producer_test.go | 28 +++++++++++++++++++++++++ 3 files changed, 41 insertions(+), 5 deletions(-) diff --git a/pulsar/internal/compression/zstd_cgo.go b/pulsar/internal/compression/zstd_cgo.go index 25429e2512..dde54ae29e 100644 --- a/pulsar/internal/compression/zstd_cgo.go +++ b/pulsar/internal/compression/zstd_cgo.go @@ -25,19 +25,23 @@ package compression import ( + "sync" + "github.com/DataDog/zstd" log "github.com/sirupsen/logrus" ) type zstdCGoProvider struct { - ctx zstd.Ctx + ctxPool sync.Pool level Level zstdLevel int } func newCGoZStdProvider(level Level) Provider { z := &zstdCGoProvider{ - ctx: zstd.NewCtx(), + ctxPool: sync.Pool{New: func() any { + return zstd.NewCtx() + }}, } switch level { @@ -61,7 +65,9 @@ func (z *zstdCGoProvider) CompressMaxSize(originalSize int) int { } func (z *zstdCGoProvider) Compress(dst, src []byte) []byte { - out, err := z.ctx.CompressLevel(dst, src, z.zstdLevel) + ctx := z.ctxPool.Get().(zstd.Ctx) + defer z.ctxPool.Put(ctx) + out, err := ctx.CompressLevel(dst, src, z.zstdLevel) if err != nil { log.WithError(err).Fatal("Failed to compress") } @@ -70,7 +76,9 @@ func (z *zstdCGoProvider) Compress(dst, src []byte) []byte { } func (z *zstdCGoProvider) Decompress(dst, src []byte, originalSize int) ([]byte, error) { - return z.ctx.Decompress(dst, src) + ctx := z.ctxPool.Get().(zstd.Ctx) + defer z.ctxPool.Put(ctx) + return ctx.Decompress(dst, src) } func (z *zstdCGoProvider) Close() error { diff --git a/pulsar/internal/utils.go b/pulsar/internal/utils.go index 9378d9dcb6..2dc8210147 100644 --- a/pulsar/internal/utils.go +++ b/pulsar/internal/utils.go @@ -40,7 +40,7 @@ func TimestampMillis(t time.Time) uint64 { // GetAndAdd perform atomic read and update func GetAndAdd(n *uint64, diff uint64) uint64 { for { - v := *n + v := atomic.LoadUint64(n) if atomic.CompareAndSwapUint64(n, v, v+diff) { return v } diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 0d74cdeefe..3b9ea7e8da 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -2357,6 +2357,34 @@ func TestFailPendingMessageWithClose(t *testing.T) { assert.Equal(t, 0, partitionProducerImp.pendingQueue.Size()) } +func TestSendConcurrently(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + assert.NoError(t, err) + defer client.Close() + testProducer, err := client.CreateProducer(ProducerOptions{ + Topic: newTopicName(), + CompressionType: ZSTD, + CompressionLevel: Better, + DisableBatching: true, + }) + assert.NoError(t, err) + + var wg sync.WaitGroup + for i := 0; i < 100; i++ { + wg.Add(1) + go func() { + _, err := testProducer.Send(context.Background(), &ProducerMessage{ + Payload: make([]byte, 100), + }) + assert.NoError(t, err) + wg.Done() + }() + } + wg.Wait() +} + type pendingQueueWrapper struct { pendingQueue internal.BlockingQueue writtenBuffers *[]internal.Buffer From 2a28e21c59d005515e118fed5bf8f333d6699e39 Mon Sep 17 00:00:00 2001 From: Jayant Date: Thu, 1 Feb 2024 21:52:52 -0500 Subject: [PATCH 318/348] [Producer] respect context cancellation in Flush (#1165) ### Motivation The producer's `Flush` method does not respect context cancellation. If the caller's context get's cancelled, it will have to wait for the producer to finish flushing. ### Modifications This change adds a `FlushWithCtx` method which takes a context and selects on two channels. --- pulsar/consumer_test.go | 6 ++--- .../producer_interceptor_test.go | 4 ++++ pulsar/producer.go | 7 ++++-- pulsar/producer_impl.go | 6 ++++- pulsar/producer_partition.go | 18 +++++++++++--- pulsar/producer_test.go | 24 +++++++++---------- pulsar/reader_test.go | 6 ++--- 7 files changed, 47 insertions(+), 24 deletions(-) diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index df70b0dd0b..d66e23765d 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -983,7 +983,7 @@ func TestConsumerBatchCumulativeAck(t *testing.T) { } wg.Wait() - err = producer.Flush() + err = producer.FlushWithCtx(context.Background()) assert.NoError(t, err) // send another batch @@ -1218,7 +1218,7 @@ func TestConsumerCompressionWithBatches(t *testing.T) { }, nil) } - producer.Flush() + producer.FlushWithCtx(context.Background()) for i := 0; i < N; i++ { msg, err := consumer.Receive(ctx) @@ -3932,7 +3932,7 @@ func runBatchIndexAckTest(t *testing.T, ackWithResponse bool, cumulative bool, o log.Printf("Sent to %v:%d:%d", id, id.BatchIdx(), id.BatchSize()) }) } - assert.Nil(t, producer.Flush()) + assert.Nil(t, producer.FlushWithCtx(context.Background())) msgIds := make([]MessageID, BatchingMaxSize) for i := 0; i < BatchingMaxSize; i++ { diff --git a/pulsar/internal/pulsartracing/producer_interceptor_test.go b/pulsar/internal/pulsartracing/producer_interceptor_test.go index 8d8e6965b8..1c2c712fcf 100644 --- a/pulsar/internal/pulsartracing/producer_interceptor_test.go +++ b/pulsar/internal/pulsartracing/producer_interceptor_test.go @@ -67,4 +67,8 @@ func (p *mockProducer) Flush() error { return nil } +func (p *mockProducer) FlushWithCtx(ctx context.Context) error { + return nil +} + func (p *mockProducer) Close() {} diff --git a/pulsar/producer.go b/pulsar/producer.go index 70d152c78b..f8013a16ff 100644 --- a/pulsar/producer.go +++ b/pulsar/producer.go @@ -237,10 +237,13 @@ type Producer interface { // return the last sequence id published by this producer. LastSequenceID() int64 - // Flush all the messages buffered in the client and wait until all messages have been successfully - // persisted. + // Deprecated: Use `FlushWithCtx()` instead. Flush() error + // Flush all the messages buffered in the client and wait until all messageshave been successfully + // persisted. + FlushWithCtx(ctx context.Context) error + // Close the producer and releases resources allocated // No more writes will be accepted from this producer. Waits until all pending write request are persisted. In case // of errors, pending writes will not be retried. diff --git a/pulsar/producer_impl.go b/pulsar/producer_impl.go index 3c45b597d0..ca923108fe 100644 --- a/pulsar/producer_impl.go +++ b/pulsar/producer_impl.go @@ -334,11 +334,15 @@ func (p *producer) LastSequenceID() int64 { } func (p *producer) Flush() error { + return p.FlushWithCtx(context.Background()) +} + +func (p *producer) FlushWithCtx(ctx context.Context) error { p.RLock() defer p.RUnlock() for _, pp := range p.producers { - if err := pp.Flush(); err != nil { + if err := pp.FlushWithCtx(ctx); err != nil { return err } diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index 1b79053e38..fbcc5b9776 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -1422,15 +1422,27 @@ func (p *partitionProducer) LastSequenceID() int64 { } func (p *partitionProducer) Flush() error { + return p.FlushWithCtx(context.Background()) +} + +func (p *partitionProducer) FlushWithCtx(ctx context.Context) error { flushReq := &flushRequest{ doneCh: make(chan struct{}), err: nil, } - p.cmdChan <- flushReq + select { + case <-ctx.Done(): + return ctx.Err() + case p.cmdChan <- flushReq: + } // wait for the flush request to complete - <-flushReq.doneCh - return flushReq.err + select { + case <-ctx.Done(): + return ctx.Err() + case <-flushReq.doneCh: + return flushReq.err + } } func (p *partitionProducer) getProducerState() producerState { diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index 3b9ea7e8da..ba5911565e 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -159,7 +159,7 @@ func TestProducerAsyncSend(t *testing.T) { assert.NoError(t, err) } - err = producer.Flush() + err = producer.FlushWithCtx(context.Background()) assert.Nil(t, err) wg.Wait() @@ -220,7 +220,7 @@ func TestProducerFlushDisableBatching(t *testing.T) { assert.NoError(t, err) } - err = producer.Flush() + err = producer.FlushWithCtx(context.Background()) assert.Nil(t, err) wg.Wait() @@ -387,7 +387,7 @@ func TestFlushInProducer(t *testing.T) { }) assert.Nil(t, err) } - err = producer.Flush() + err = producer.FlushWithCtx(context.Background()) assert.Nil(t, err) wg.Wait() @@ -429,7 +429,7 @@ func TestFlushInProducer(t *testing.T) { assert.Nil(t, err) } - err = producer.Flush() + err = producer.FlushWithCtx(context.Background()) assert.Nil(t, err) wg.Wait() @@ -500,7 +500,7 @@ func TestFlushInPartitionedProducer(t *testing.T) { } // After flush, should be able to consume. - err = producer.Flush() + err = producer.FlushWithCtx(context.Background()) assert.Nil(t, err) wg.Wait() @@ -1717,7 +1717,7 @@ func TestMultipleSchemaOfKeyBasedBatchProducerConsumer(t *testing.T) { } } - producer.Flush() + producer.FlushWithCtx(context.Background()) //// create consumer consumer, err := client.Subscribe(ConsumerOptions{ @@ -1808,7 +1808,7 @@ func TestMultipleSchemaProducerConsumer(t *testing.T) { assert.NotNil(t, id) }) } - producer.Flush() + producer.FlushWithCtx(context.Background()) //// create consumer consumer, err := client.Subscribe(ConsumerOptions{ @@ -2027,9 +2027,9 @@ func TestMemLimitRejectProducerMessages(t *testing.T) { assert.ErrorContains(t, err, getResultStr(ClientMemoryBufferIsFull)) // flush pending msg - err = producer1.Flush() + err = producer1.FlushWithCtx(context.Background()) assert.NoError(t, err) - err = producer2.Flush() + err = producer2.FlushWithCtx(context.Background()) assert.NoError(t, err) assert.Equal(t, int64(0), c.(*client).memLimit.CurrentUsage()) @@ -2118,9 +2118,9 @@ func TestMemLimitRejectProducerMessagesWithSchema(t *testing.T) { assert.ErrorContains(t, err, getResultStr(ClientMemoryBufferIsFull)) // flush pending msg - err = producer1.Flush() + err = producer1.FlushWithCtx(context.Background()) assert.NoError(t, err) - err = producer2.Flush() + err = producer2.FlushWithCtx(context.Background()) assert.NoError(t, err) assert.Equal(t, int64(0), c.(*client).memLimit.CurrentUsage()) @@ -2244,7 +2244,7 @@ func TestMemLimitContextCancel(t *testing.T) { cancel() wg.Wait() - err = producer.Flush() + err = producer.FlushWithCtx(context.Background()) assert.NoError(t, err) assert.Equal(t, int64(0), c.(*client).memLimit.CurrentUsage()) diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index ec10f8f162..c8228a7ca9 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -277,7 +277,7 @@ func TestReaderOnSpecificMessageWithBatching(t *testing.T) { }) } - err = producer.Flush() + err = producer.FlushWithCtx(context.Background()) assert.NoError(t, err) // create reader on 5th message (not included) @@ -353,7 +353,7 @@ func TestReaderOnLatestWithBatching(t *testing.T) { }) } - err = producer.Flush() + err = producer.FlushWithCtx(context.Background()) assert.NoError(t, err) // create reader on 5th message (not included) @@ -592,7 +592,7 @@ func TestReaderSeek(t *testing.T) { seekID = id } } - err = producer.Flush() + err = producer.FlushWithCtx(context.Background()) assert.NoError(t, err) for i := 0; i < N; i++ { From f476814a9e1bac484e9589d4fb3f299066f54ddc Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 6 Feb 2024 19:24:55 -0800 Subject: [PATCH 319/348] Added CodeQL static code scanner (#1169) * Added CodeQL static code scanner * Update codeql.yml --- .github/workflows/codeql.yml | 90 ++++++++++++++++++++++++++++++++++++ 1 file changed, 90 insertions(+) create mode 100644 .github/workflows/codeql.yml diff --git a/.github/workflows/codeql.yml b/.github/workflows/codeql.yml new file mode 100644 index 0000000000..869a19ad80 --- /dev/null +++ b/.github/workflows/codeql.yml @@ -0,0 +1,90 @@ +# 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. + +name: "CodeQL" + +on: + push: + branches: [ "master" ] + pull_request: + branches: [ "master" ] + schedule: + - cron: '43 13 * * 0' + +jobs: + analyze: + name: Analyze + # Runner size impacts CodeQL analysis time. To learn more, please see: + # - https://gh.io/recommended-hardware-resources-for-running-codeql + # - https://gh.io/supported-runners-and-hardware-resources + # - https://gh.io/using-larger-runners + # Consider using larger runners for possible analysis time improvements. + runs-on: ${{ (matrix.language == 'swift' && 'macos-latest') || 'ubuntu-latest' }} + timeout-minutes: ${{ (matrix.language == 'swift' && 120) || 360 }} + permissions: + # required for all workflows + security-events: write + + # only required for workflows in private repositories + actions: read + contents: read + + strategy: + fail-fast: false + matrix: + language: [ 'go' ] + # CodeQL supports [ 'c-cpp', 'csharp', 'go', 'java-kotlin', 'javascript-typescript', 'python', 'ruby', 'swift' ] + # Use only 'java-kotlin' to analyze code written in Java, Kotlin or both + # Use only 'javascript-typescript' to analyze code written in JavaScript, TypeScript or both + # Learn more about CodeQL language support at https://aka.ms/codeql-docs/language-support + + steps: + - name: Checkout repository + uses: actions/checkout@v4 + + # Initializes the CodeQL tools for scanning. + - name: Initialize CodeQL + uses: github/codeql-action/init@v3 + with: + languages: ${{ matrix.language }} + # If you wish to specify custom queries, you can do so here or in a config file. + # By default, queries listed here will override any specified in a config file. + # Prefix the list here with "+" to use these queries and those in the config file. + + # For more details on CodeQL's query packs, refer to: https://docs.github.com/en/code-security/code-scanning/automatically-scanning-your-code-for-vulnerabilities-and-errors/configuring-code-scanning#using-queries-in-ql-packs + # queries: security-extended,security-and-quality + + + # Autobuild attempts to build any compiled languages (C/C++, C#, Go, Java, or Swift). + # If this step fails, then you should remove it and run the build manually (see below) + - name: Autobuild + uses: github/codeql-action/autobuild@v3 + + # ℹ️ Command-line programs to run using the OS shell. + # 📚 See https://docs.github.com/en/actions/using-workflows/workflow-syntax-for-github-actions#jobsjob_idstepsrun + + # If the Autobuild fails above, remove it and uncomment the following three lines. + # modify them (or add more) to build your code if your project, please refer to the EXAMPLE below for guidance. + + # - run: | + # echo "Run, Build Application using script" + # ./location_of_script_within_repo/buildscript.sh + + - name: Perform CodeQL Analysis + uses: github/codeql-action/analyze@v3 + with: + category: "/language:${{matrix.language}}" From c2ca7e81f0c609cd8fb7b13695664519e63e4501 Mon Sep 17 00:00:00 2001 From: Peter Hull <56369394+petermnhull@users.noreply.github.com> Date: Tue, 20 Feb 2024 02:31:00 +0000 Subject: [PATCH 320/348] [Fix] Fix Bytes Schema (#1173) --- pulsar/schema.go | 2 ++ pulsar/schema_test.go | 64 +++++++++++++++++++++++++++++++++++++++ pulsar/table_view_test.go | 7 +++++ 3 files changed, 73 insertions(+) diff --git a/pulsar/schema.go b/pulsar/schema.go index fd9d412dc6..3427fb263a 100644 --- a/pulsar/schema.go +++ b/pulsar/schema.go @@ -93,6 +93,8 @@ func NewSchema(schemaType SchemaType, schemaData []byte, properties map[string]s var schemaDef = string(schemaData) var s Schema switch schemaType { + case BYTES: + s = NewBytesSchema(properties) case STRING: s = NewStringSchema(properties) case JSON: diff --git a/pulsar/schema_test.go b/pulsar/schema_test.go index c2008f6de9..34216c4779 100644 --- a/pulsar/schema_test.go +++ b/pulsar/schema_test.go @@ -19,11 +19,14 @@ package pulsar import ( "context" + "fmt" "log" "testing" + "time" pb "github.com/apache/pulsar-client-go/integration-tests/pb" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) type testJSON struct { @@ -55,6 +58,67 @@ func createClient() Client { return client } +func TestBytesSchema(t *testing.T) { + client := createClient() + defer client.Close() + + topic := newTopicName() + + properties := make(map[string]string) + properties["pulsar"] = "hello" + producerSchemaBytes := NewBytesSchema(properties) + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + Schema: producerSchemaBytes, + }) + assert.NoError(t, err) + + _, err = producer.Send(context.Background(), &ProducerMessage{ + Value: []byte(`{"key": "value"}`), + }) + require.NoError(t, err) + _, err = producer.Send(context.Background(), &ProducerMessage{ + Value: []byte(`something else`), + }) + require.NoError(t, err) + producer.Close() + + // Create consumer + consumerSchemaBytes := NewBytesSchema(nil) + assert.NotNil(t, consumerSchemaBytes) + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "sub-1", + Schema: consumerSchemaBytes, + SubscriptionInitialPosition: SubscriptionPositionEarliest, + }) + assert.Nil(t, err) + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) + defer cancel() + + // Receive first message + var out1 []byte + msg1, err := consumer.Receive(ctx) + assert.NoError(t, err) + err = msg1.GetSchemaValue(&out1) + assert.NoError(t, err) + assert.Equal(t, []byte(`{"key": "value"}`), out1) + consumer.Ack(msg1) + require.NoError(t, err) + + // Receive second message + var out2 []byte + msg2, err := consumer.Receive(ctx) + fmt.Println(string(msg2.Payload())) + assert.NoError(t, err) + err = msg2.GetSchemaValue(&out2) + assert.NoError(t, err) + assert.Equal(t, []byte(`something else`), out2) + + defer consumer.Close() +} + func TestJsonSchema(t *testing.T) { client := createClient() defer client.Close() diff --git a/pulsar/table_view_test.go b/pulsar/table_view_test.go index 45b9441169..2368e3d846 100644 --- a/pulsar/table_view_test.go +++ b/pulsar/table_view_test.go @@ -90,6 +90,13 @@ func TestTableViewSchemas(t *testing.T) { expValueOut interface{} valueCheck func(t *testing.T, got interface{}) // Overrides expValueOut for more complex checks }{ + { + name: "BytesSchema", + schema: NewBytesSchema(nil), + schemaType: []byte(`any`), + producerValue: []byte(`hello pulsar`), + expValueOut: []byte(`hello pulsar`), + }, { name: "StringSchema", schema: NewStringSchema(nil), From 3b9b1f8895d8924ec98db4612806b9871f1d135b Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Fri, 23 Feb 2024 15:49:08 +0800 Subject: [PATCH 321/348] [feat] Support partitioned topic reader (#1178) Master Issue: #1177 ### Motivation Currently, there is an issue with the reader implementation. If the reader is creating, it won't get the topic metadata from the topic. The reader can only read messages from a single topic. If the topic is a partitioned topic, the reader won't know that and will try to create a non-partition topic with the same name. And it will lead to this issue: /~https://github.com/apache/pulsar/issues/22032 ### Modifications - Support partitioned topic reader --- pulsar/consumer.go | 7 +++ pulsar/consumer_impl.go | 47 ++++++++++++++- pulsar/consumer_partition.go | 36 +++++++++++ pulsar/reader.go | 1 + pulsar/reader_impl.go | 112 +++++++++++++++-------------------- pulsar/reader_test.go | 100 +++++++++++++++++++++++++++---- 6 files changed, 227 insertions(+), 76 deletions(-) diff --git a/pulsar/consumer.go b/pulsar/consumer.go index 667bff66cd..fea94cf6a3 100644 --- a/pulsar/consumer.go +++ b/pulsar/consumer.go @@ -246,6 +246,13 @@ type ConsumerOptions struct { // SubscriptionMode specifies the subscription mode to be used when subscribing to a topic. // Default is `Durable` SubscriptionMode SubscriptionMode + + // StartMessageIDInclusive, if true, the consumer will start at the `StartMessageID`, included. + // Default is `false` and the consumer will start from the "next" message + StartMessageIDInclusive bool + + // startMessageID specifies the message id to start from. Currently, it's only used for the reader internally. + startMessageID *trackingMessageID } // Consumer is an interface that abstracts behavior of Pulsar's consumer diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index 75d839b412..0c31a1aafc 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -384,7 +384,8 @@ func (c *consumer) internalTopicSubscribeToPartitions() error { metadata: metadata, subProperties: subProperties, replicateSubscriptionState: c.options.ReplicateSubscriptionState, - startMessageID: nil, + startMessageID: c.options.startMessageID, + startMessageIDInclusive: c.options.StartMessageIDInclusive, subscriptionMode: c.options.SubscriptionMode, readCompacted: c.options.ReadCompacted, interceptors: c.options.Interceptors, @@ -707,6 +708,50 @@ func (c *consumer) checkMsgIDPartition(msgID MessageID) error { return nil } +func (c *consumer) hasNext() bool { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() // Make sure all paths cancel the context to avoid context leak + + var wg sync.WaitGroup + wg.Add(len(c.consumers)) + + hasNext := make(chan bool) + for _, pc := range c.consumers { + pc := pc + go func() { + defer wg.Done() + if pc.hasNext() { + select { + case hasNext <- true: + case <-ctx.Done(): + } + } + }() + } + + go func() { + wg.Wait() + close(hasNext) // Close the channel after all goroutines have finished + }() + + // Wait for either a 'true' result or for all goroutines to finish + for hn := range hasNext { + if hn { + return true + } + } + + return false +} + +func (c *consumer) setLastDequeuedMsg(msgID MessageID) error { + if err := c.checkMsgIDPartition(msgID); err != nil { + return err + } + c.consumers[msgID.PartitionIdx()].lastDequeuedMsg = toTrackingMessageID(msgID) + return nil +} + var r = &random{ R: rand.New(rand.NewSource(time.Now().UnixNano())), } diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index fd6441c1cb..95b5bc0946 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -174,6 +174,8 @@ type partitionConsumer struct { chunkedMsgCtxMap *chunkedMsgCtxMap unAckChunksTracker *unAckChunksTracker ackGroupingTracker ackGroupingTracker + + lastMessageInBroker *trackingMessageID } func (pc *partitionConsumer) ActiveConsumerChanged(isActive bool) { @@ -1970,6 +1972,40 @@ func (pc *partitionConsumer) discardCorruptedMessage(msgID *pb.MessageIdData, pc.availablePermits.inc() } +func (pc *partitionConsumer) hasNext() bool { + if pc.lastMessageInBroker != nil && pc.hasMoreMessages() { + return true + } + + for { + lastMsgID, err := pc.getLastMessageID() + if err != nil { + pc.log.WithError(err).Error("Failed to get last message id from broker") + continue + } else { + pc.lastMessageInBroker = lastMsgID + break + } + } + + return pc.hasMoreMessages() +} + +func (pc *partitionConsumer) hasMoreMessages() bool { + if pc.lastDequeuedMsg != nil { + return pc.lastMessageInBroker.isEntryIDValid() && pc.lastMessageInBroker.greater(pc.lastDequeuedMsg.messageID) + } + + if pc.options.startMessageIDInclusive { + return pc.lastMessageInBroker.isEntryIDValid() && + pc.lastMessageInBroker.greaterEqual(pc.startMessageID.get().messageID) + } + + // Non-inclusive + return pc.lastMessageInBroker.isEntryIDValid() && + pc.lastMessageInBroker.greater(pc.startMessageID.get().messageID) +} + // _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) { diff --git a/pulsar/reader.go b/pulsar/reader.go index 5e1a73b988..1c5235d422 100644 --- a/pulsar/reader.go +++ b/pulsar/reader.go @@ -136,5 +136,6 @@ type Reader interface { SeekByTime(time time.Time) error // GetLastMessageID get the last message id available for consume. + // It only works for single topic reader. It will return an error when the reader is the multi-topic reader. GetLastMessageID() (MessageID, error) } diff --git a/pulsar/reader_impl.go b/pulsar/reader_impl.go index 7b260b88db..bf91c67fa5 100644 --- a/pulsar/reader_impl.go +++ b/pulsar/reader_impl.go @@ -34,12 +34,11 @@ const ( type reader struct { sync.Mutex - client *client - pc *partitionConsumer - messageCh chan ConsumerMessage - lastMessageInBroker *trackingMessageID - log log.Logger - metrics *internal.LeveledMetrics + client *client + messageCh chan ConsumerMessage + log log.Logger + metrics *internal.LeveledMetrics + c *consumer } func newReader(client *client, options ReaderOptions) (Reader, error) { @@ -98,25 +97,25 @@ func newReader(client *client, options ReaderOptions) (Reader, error) { options.ExpireTimeOfIncompleteChunk = time.Minute } - consumerOptions := &partitionConsumerOpts{ - topic: options.Topic, - consumerName: options.Name, - subscription: subscriptionName, - subscriptionType: Exclusive, - receiverQueueSize: receiverQueueSize, + consumerOptions := &ConsumerOptions{ + Topic: options.Topic, + Name: options.Name, + SubscriptionName: subscriptionName, + Type: Exclusive, + ReceiverQueueSize: receiverQueueSize, + SubscriptionMode: NonDurable, + ReadCompacted: options.ReadCompacted, + Properties: options.Properties, + NackRedeliveryDelay: defaultNackRedeliveryDelay, + ReplicateSubscriptionState: false, + Decryption: options.Decryption, + Schema: options.Schema, + BackoffPolicy: options.BackoffPolicy, + MaxPendingChunkedMessage: options.MaxPendingChunkedMessage, + ExpireTimeOfIncompleteChunk: options.ExpireTimeOfIncompleteChunk, + AutoAckIncompleteChunk: options.AutoAckIncompleteChunk, startMessageID: startMessageID, - startMessageIDInclusive: options.StartMessageIDInclusive, - subscriptionMode: NonDurable, - readCompacted: options.ReadCompacted, - metadata: options.Properties, - nackRedeliveryDelay: defaultNackRedeliveryDelay, - replicateSubscriptionState: false, - decryption: options.Decryption, - schema: options.Schema, - backoffPolicy: options.BackoffPolicy, - maxPendingChunkedMessage: options.MaxPendingChunkedMessage, - expireTimeOfIncompleteChunk: options.ExpireTimeOfIncompleteChunk, - autoAckIncompleteChunk: options.AutoAckIncompleteChunk, + StartMessageIDInclusive: options.StartMessageIDInclusive, } reader := &reader{ @@ -131,20 +130,25 @@ func newReader(client *client, options ReaderOptions) (Reader, error) { if err != nil { return nil, err } + // Provide dummy rlq router with not dlq policy + rlq, err := newRetryRouter(client, nil, false, client.log) + if err != nil { + return nil, err + } - pc, err := newPartitionConsumer(nil, client, consumerOptions, reader.messageCh, dlq, reader.metrics) + c, err := newInternalConsumer(client, *consumerOptions, options.Topic, reader.messageCh, dlq, rlq, false) if err != nil { close(reader.messageCh) return nil, err } + reader.c = c - reader.pc = pc reader.metrics.ReadersOpened.Inc() return reader, nil } func (r *reader) Topic() string { - return r.pc.topic + return r.c.topic } func (r *reader) Next(ctx context.Context) (Message, error) { @@ -158,9 +162,14 @@ func (r *reader) Next(ctx context.Context) (Message, error) { // Acknowledge message immediately because the reader is based on non-durable subscription. When it reconnects, // it will specify the subscription position anyway msgID := cm.Message.ID() - mid := toTrackingMessageID(msgID) - r.pc.lastDequeuedMsg = mid - r.pc.AckID(mid) + err := r.c.setLastDequeuedMsg(msgID) + if err != nil { + return nil, err + } + err = r.c.AckID(msgID) + if err != nil { + return nil, err + } return cm.Message, nil case <-ctx.Done(): return nil, ctx.Err() @@ -169,41 +178,11 @@ func (r *reader) Next(ctx context.Context) (Message, error) { } func (r *reader) HasNext() bool { - if r.lastMessageInBroker != nil && r.hasMoreMessages() { - return true - } - - for { - lastMsgID, err := r.pc.getLastMessageID() - if err != nil { - r.log.WithError(err).Error("Failed to get last message id from broker") - continue - } else { - r.lastMessageInBroker = lastMsgID - break - } - } - - return r.hasMoreMessages() -} - -func (r *reader) hasMoreMessages() bool { - if r.pc.lastDequeuedMsg != nil { - return r.lastMessageInBroker.isEntryIDValid() && r.lastMessageInBroker.greater(r.pc.lastDequeuedMsg.messageID) - } - - if r.pc.options.startMessageIDInclusive { - return r.lastMessageInBroker.isEntryIDValid() && - r.lastMessageInBroker.greaterEqual(r.pc.startMessageID.get().messageID) - } - - // Non-inclusive - return r.lastMessageInBroker.isEntryIDValid() && - r.lastMessageInBroker.greater(r.pc.startMessageID.get().messageID) + return r.c.hasNext() } func (r *reader) Close() { - r.pc.Close() + r.c.Close() r.client.handlers.Del(r) r.metrics.ReadersClosed.Inc() } @@ -235,16 +214,19 @@ func (r *reader) Seek(msgID MessageID) error { return nil } - return r.pc.Seek(mid) + return r.c.Seek(mid) } func (r *reader) SeekByTime(time time.Time) error { r.Lock() defer r.Unlock() - return r.pc.SeekByTime(time) + return r.c.SeekByTime(time) } func (r *reader) GetLastMessageID() (MessageID, error) { - return r.pc.getLastMessageID() + if len(r.c.consumers) > 1 { + return nil, fmt.Errorf("GetLastMessageID is not supported for multi-topics reader") + } + return r.c.consumers[0].getLastMessageID() } diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index c8228a7ca9..ccf52875ba 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -24,6 +24,9 @@ import ( "time" "github.com/apache/pulsar-client-go/pulsar/crypto" + "github.com/apache/pulsar-client-go/pulsaradmin" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" "github.com/google/uuid" "github.com/stretchr/testify/assert" ) @@ -90,10 +93,10 @@ func TestReaderConfigChunk(t *testing.T) { defer r1.Close() // verify specified chunk options - pcOpts := r1.(*reader).pc.options - assert.Equal(t, 50, pcOpts.maxPendingChunkedMessage) - assert.Equal(t, 30*time.Second, pcOpts.expireTimeOfIncompleteChunk) - assert.True(t, pcOpts.autoAckIncompleteChunk) + pcOpts := r1.(*reader).c.options + assert.Equal(t, 50, pcOpts.MaxPendingChunkedMessage) + assert.Equal(t, 30*time.Second, pcOpts.ExpireTimeOfIncompleteChunk) + assert.True(t, pcOpts.AutoAckIncompleteChunk) r2, err := client.CreateReader(ReaderOptions{ Topic: "my-topic2", @@ -103,10 +106,10 @@ func TestReaderConfigChunk(t *testing.T) { defer r2.Close() // verify default chunk options - pcOpts = r2.(*reader).pc.options - assert.Equal(t, 100, pcOpts.maxPendingChunkedMessage) - assert.Equal(t, time.Minute, pcOpts.expireTimeOfIncompleteChunk) - assert.False(t, pcOpts.autoAckIncompleteChunk) + pcOpts = r2.(*reader).c.options + assert.Equal(t, 100, pcOpts.MaxPendingChunkedMessage) + assert.Equal(t, time.Minute, pcOpts.ExpireTimeOfIncompleteChunk) + assert.False(t, pcOpts.AutoAckIncompleteChunk) } func TestReader(t *testing.T) { @@ -153,6 +156,50 @@ func TestReader(t *testing.T) { } } +func TestReaderOnPartitionedTopic(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + + topic := newTopicName() + assert.Nil(t, createPartitionedTopic(topic, 3)) + ctx := context.Background() + // create reader + reader, err := client.CreateReader(ReaderOptions{ + Topic: topic, + StartMessageID: EarliestMessageID(), + }) + assert.Nil(t, err) + defer reader.Close() + + // create producer + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + }) + assert.Nil(t, err) + defer producer.Close() + + // send 10 messages + for i := 0; i < 10; i++ { + _, err := producer.Send(ctx, &ProducerMessage{ + Payload: []byte(fmt.Sprintf("hello-%d", i)), + }) + assert.NoError(t, err) + } + + // receive 10 messages + for i := 0; i < 10; i++ { + msg, err := reader.Next(context.Background()) + assert.NoError(t, err) + + expectMsg := fmt.Sprintf("hello-%d", i) + assert.Equal(t, []byte(expectMsg), msg.Payload()) + } +} + func TestReaderConnectError(t *testing.T) { client, err := NewClient(ClientOptions{ URL: "pulsar://invalid-hostname:6650", @@ -422,7 +469,6 @@ func TestReaderHasNext(t *testing.T) { assert.NotNil(t, msgID) } - // create reader on 5th message (not included) reader, err := client.CreateReader(ReaderOptions{ Topic: topic, StartMessageID: EarliestMessageID(), @@ -880,7 +926,7 @@ func TestReaderWithBackoffPolicy(t *testing.T) { assert.NotNil(t, _reader) assert.Nil(t, err) - partitionConsumerImp := _reader.(*reader).pc + partitionConsumerImp := _reader.(*reader).c.consumers[0] // 1 s startTime := time.Now() partitionConsumerImp.reconnectToBroker() @@ -943,3 +989,37 @@ func TestReaderGetLastMessageID(t *testing.T) { assert.Equal(t, lastMsgID.LedgerID(), getLastMessageID.LedgerID()) assert.Equal(t, lastMsgID.EntryID(), getLastMessageID.EntryID()) } + +func TestReaderGetLastMessageIDOnMultiTopics(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: serviceURL, + }) + assert.Nil(t, err) + topic := newTopicName() + assert.Nil(t, createPartitionedTopic(topic, 3)) + + reader, err := client.CreateReader(ReaderOptions{ + Topic: topic, + StartMessageID: EarliestMessageID(), + }) + assert.Nil(t, err) + _, err = reader.GetLastMessageID() + assert.NotNil(t, err) +} + +func createPartitionedTopic(topic string, n int) error { + admin, err := pulsaradmin.NewClient(&config.Config{}) + if err != nil { + return err + } + + topicName, err := utils.GetTopicName(topic) + if err != nil { + return err + } + err = admin.Topics().Create(*topicName, n) + if err != nil { + return err + } + return nil +} From 5d258272cb83444fe156dcbb57cbf8f2d475a50b Mon Sep 17 00:00:00 2001 From: Jinjun Pan <75996911+panszobe@users.noreply.github.com> Date: Fri, 23 Feb 2024 21:47:37 +0800 Subject: [PATCH 322/348] [Fix] Fix available permits in MessageReceived (#1181) Fixes #1180 ### Motivation In the `MessageReceived`, the number of skipped messages should be increased to available permits to avoid skipped permits leading flow request not be sent. --------- Co-authored-by: panjinjun <1619-panjinjun@users.noreply.git.sysop.bigo.sg> --- pulsar/consumer_partition.go | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 95b5bc0946..3572a52269 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -1093,7 +1093,10 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header pc.metrics.MessagesReceived.Add(float64(numMsgs)) pc.metrics.PrefetchedMessages.Add(float64(numMsgs)) - var bytesReceived int + var ( + bytesReceived int + skippedMessages int32 + ) for i := 0; i < numMsgs; i++ { smm, payload, err := reader.ReadMessage() if err != nil || payload == nil { @@ -1102,6 +1105,7 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header } if ackSet != nil && !ackSet.Test(uint(i)) { pc.log.Debugf("Ignoring message from %vth message, which has been acknowledged", i) + skippedMessages++ continue } @@ -1120,6 +1124,7 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header if pc.messageShouldBeDiscarded(trackingMsgID) { pc.AckID(trackingMsgID) + skippedMessages++ continue } @@ -1144,6 +1149,7 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header } if pc.ackGroupingTracker.isDuplicate(msgID) { + skippedMessages++ continue } @@ -1218,6 +1224,10 @@ func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, header pc.markScaleIfNeed() } + if skippedMessages > 0 { + pc.availablePermits.add(skippedMessages) + } + // send messages to the dispatcher pc.queueCh <- messages return nil From a881240db862e48f1791575aecb10ecc149bfa8b Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Tue, 27 Feb 2024 00:59:34 +0800 Subject: [PATCH 323/348] fix: make function state values `omitempty` (#1185) * make function state values omitempty * fix --- pulsaradmin/pkg/utils/function_state.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsaradmin/pkg/utils/function_state.go b/pulsaradmin/pkg/utils/function_state.go index 63fa15057c..c8a8c4b8cb 100644 --- a/pulsaradmin/pkg/utils/function_state.go +++ b/pulsaradmin/pkg/utils/function_state.go @@ -19,8 +19,8 @@ package utils type FunctionState struct { Key string `json:"key"` - StringValue string `json:"stringValue"` - ByteValue []byte `json:"byteValue"` - NumValue int64 `json:"numberValue"` - Version int64 `json:"version"` + StringValue string `json:"stringValue,omitempty"` + ByteValue []byte `json:"byteValue,omitempty"` + NumValue int64 `json:"numberValue,omitempty"` + Version int64 `json:"version,omitempty"` } From 88a8d85cf6d6a4f282a5b39a2140a7bb06ba0f3b Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 28 Feb 2024 18:39:00 +0800 Subject: [PATCH 324/348] [fix] Fix Infinite Loop in Reader's `HasNext` Function (#1182) Fixes #1171 ### Motivation If `getLastMessageId` continually fails, the reader.HasNext can get stuck in an infinite loop. Without any backoff, the reader would keep trying forever. ### Modifications - Implemented a backoff policy for `getLastMessageID`. - If HasNext fails, it now returns false. #### Should the reader.HasNext returned `false` in case of failure? Currently, the `HasNext` method doesn't report errors. However, failure is still possible. For instance, if `getLastMessageID` repeatedly fails and hits the retry limit. An option is to keep trying forever, but this would stall all user code. This isn't user-friendly, so I rejected this solution. #### Couldn't utilize the BackOffPolicy in the Reader Options The `HasNext` retry mechanism requires to use of `IsMaxBackoffReached` for the backoff. But it isn't exposed in the `BackOffPolicy` interface. Introducing a new method to the `BackOffPolicy` would introduce breaking changes for the user backoff implementation. So, I choose not to implement it. Before we do it, we need to refine the BackOffPolicy. --- pulsar/client_impl.go | 24 +++++++------- pulsar/consumer_partition.go | 53 ++++++++++++++++++++--------- pulsar/reader.go | 1 + pulsar/reader_test.go | 64 ++++++++++++++++++++++++++++++++++++ 4 files changed, 115 insertions(+), 27 deletions(-) diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index 7daf6f62ab..65aed3b963 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -40,14 +40,15 @@ const ( ) type client struct { - cnxPool internal.ConnectionPool - rpcClient internal.RPCClient - handlers internal.ClientHandlers - lookupService internal.LookupService - metrics *internal.Metrics - tcClient *transactionCoordinatorClient - memLimit internal.MemoryLimitController - closeOnce sync.Once + cnxPool internal.ConnectionPool + rpcClient internal.RPCClient + handlers internal.ClientHandlers + lookupService internal.LookupService + metrics *internal.Metrics + tcClient *transactionCoordinatorClient + memLimit internal.MemoryLimitController + closeOnce sync.Once + operationTimeout time.Duration log log.Logger } @@ -161,9 +162,10 @@ func newClient(options ClientOptions) (Client, error) { c := &client{ cnxPool: internal.NewConnectionPool(tlsConfig, authProvider, connectionTimeout, keepAliveInterval, maxConnectionsPerHost, logger, metrics, connectionMaxIdleTime), - log: logger, - metrics: metrics, - memLimit: internal.NewMemoryLimitController(memLimitBytes, defaultMemoryLimitTriggerThreshold), + log: logger, + metrics: metrics, + memLimit: internal.NewMemoryLimitController(memLimitBytes, defaultMemoryLimitTriggerThreshold), + operationTimeout: operationTimeout, } serviceNameResolver := internal.NewPulsarServiceNameResolver(url) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 3572a52269..162565b2a9 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -570,15 +570,41 @@ func (pc *partitionConsumer) internalUnsubscribe(unsub *unsubscribeRequest) { func (pc *partitionConsumer) getLastMessageID() (*trackingMessageID, error) { if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing { - pc.log.WithField("state", state).Error("Failed to redeliver closing or closed consumer") - return nil, errors.New("failed to redeliver closing or closed consumer") + pc.log.WithField("state", state).Error("Failed to getLastMessageID for the closing or closed consumer") + return nil, errors.New("failed to getLastMessageID for the closing or closed consumer") } - req := &getLastMsgIDRequest{doneCh: make(chan struct{})} - pc.eventsCh <- req + remainTime := pc.client.operationTimeout + var backoff internal.BackoffPolicy + if pc.options.backoffPolicy != nil { + backoff = pc.options.backoffPolicy + } else { + backoff = &internal.DefaultBackoff{} + } + request := func() (*trackingMessageID, error) { + req := &getLastMsgIDRequest{doneCh: make(chan struct{})} + pc.eventsCh <- req - // wait for the request to complete - <-req.doneCh - return req.msgID, req.err + // wait for the request to complete + <-req.doneCh + return req.msgID, req.err + } + for { + msgID, err := request() + if err == nil { + return msgID, nil + } + if remainTime <= 0 { + pc.log.WithError(err).Error("Failed to getLastMessageID") + return nil, fmt.Errorf("failed to getLastMessageID due to %w", err) + } + nextDelay := backoff.Next() + if nextDelay > remainTime { + nextDelay = remainTime + } + remainTime -= nextDelay + pc.log.WithError(err).Errorf("Failed to get last message id from broker, retrying in %v...", nextDelay) + time.Sleep(nextDelay) + } } func (pc *partitionConsumer) internalGetLastMessageID(req *getLastMsgIDRequest) { @@ -1987,16 +2013,11 @@ func (pc *partitionConsumer) hasNext() bool { return true } - for { - lastMsgID, err := pc.getLastMessageID() - if err != nil { - pc.log.WithError(err).Error("Failed to get last message id from broker") - continue - } else { - pc.lastMessageInBroker = lastMsgID - break - } + lastMsgID, err := pc.getLastMessageID() + if err != nil { + return false } + pc.lastMessageInBroker = lastMsgID return pc.hasMoreMessages() } diff --git a/pulsar/reader.go b/pulsar/reader.go index 1c5235d422..4daa889062 100644 --- a/pulsar/reader.go +++ b/pulsar/reader.go @@ -113,6 +113,7 @@ type Reader interface { Next(context.Context) (Message, error) // HasNext checks if there is any message available to read from the current position + // If there is any errors, it will return false HasNext() bool // Close the reader and stop the broker to push more messages diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index ccf52875ba..78c222dac7 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -28,6 +28,7 @@ import ( "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" "github.com/google/uuid" + "github.com/pkg/errors" "github.com/stretchr/testify/assert" ) @@ -1023,3 +1024,66 @@ func createPartitionedTopic(topic string, n int) error { } return nil } + +func TestReaderHasNextFailed(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: serviceURL, + }) + assert.Nil(t, err) + topic := newTopicName() + r, err := client.CreateReader(ReaderOptions{ + Topic: topic, + StartMessageID: EarliestMessageID(), + }) + assert.Nil(t, err) + r.(*reader).c.consumers[0].state.Store(consumerClosing) + assert.False(t, r.HasNext()) +} + +func TestReaderHasNextRetryFailed(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: serviceURL, + OperationTimeout: 2 * time.Second, + }) + assert.Nil(t, err) + topic := newTopicName() + r, err := client.CreateReader(ReaderOptions{ + Topic: topic, + StartMessageID: EarliestMessageID(), + }) + assert.Nil(t, err) + + c := make(chan interface{}) + defer close(c) + + // Close the consumer events loop and assign a mock eventsCh + pc := r.(*reader).c.consumers[0] + pc.Close() + pc.state.Store(consumerReady) + pc.eventsCh = c + + go func() { + for e := range c { + req, ok := e.(*getLastMsgIDRequest) + assert.True(t, ok, "unexpected event type") + req.err = errors.New("expected error") + close(req.doneCh) + } + }() + minTimer := time.NewTimer(1 * time.Second) // Timer to check if r.HasNext() blocked for at least 1s + maxTimer := time.NewTimer(3 * time.Second) // Timer to ensure r.HasNext() doesn't block for more than 3s + done := make(chan bool) + go func() { + assert.False(t, r.HasNext()) + done <- true + }() + + select { + case <-maxTimer.C: + t.Fatal("r.HasNext() blocked for more than 3s") + case <-done: + assert.False(t, minTimer.Stop(), "r.HasNext() did not block for at least 1s") + assert.True(t, maxTimer.Stop()) + } + +} From c4f47abd7c82402e7d4e95b55b24ebfef4bdb029 Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Tue, 5 Mar 2024 18:40:40 +0800 Subject: [PATCH 325/348] [Improve] Add optional parameters for getPartitionedStats (#1193) ### Motivation To keep consistent with the Java client. Releted PR: /~https://github.com/apache/pulsar/pull/21611 ### Modifications Add `GetStatsOptions` params. --- Makefile | 2 +- pulsar/consumer_test.go | 12 +-- pulsaradmin/pkg/admin/topic.go | 34 ++++++ pulsaradmin/pkg/admin/topic_test.go | 160 ++++++++++++++++++++++++++++ pulsaradmin/pkg/utils/data.go | 8 ++ 5 files changed, 209 insertions(+), 7 deletions(-) diff --git a/Makefile b/Makefile index 4eb590b06b..d044237221 100644 --- a/Makefile +++ b/Makefile @@ -18,7 +18,7 @@ # IMAGE_NAME = pulsar-client-go-test:latest -PULSAR_VERSION ?= 2.10.3 +PULSAR_VERSION ?= 3.2.0 PULSAR_IMAGE = apachepulsar/pulsar:$(PULSAR_VERSION) GO_VERSION ?= 1.18 GOLANG_IMAGE = golang:$(GO_VERSION) diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index d66e23765d..4a3b532d05 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -2219,6 +2219,12 @@ func TestConsumerAddTopicPartitions(t *testing.T) { assert.Nil(t, err) defer producer.Close() + // Increase number of partitions to 10 + makeHTTPCall(t, http.MethodPost, testURL, "10") + + // Wait for the producer/consumers to pick up the change + time.Sleep(1 * time.Second) + consumer, err := client.Subscribe(ConsumerOptions{ Topic: topic, SubscriptionName: "my-sub", @@ -2227,12 +2233,6 @@ func TestConsumerAddTopicPartitions(t *testing.T) { assert.Nil(t, err) defer consumer.Close() - // Increase number of partitions to 10 - makeHTTPCall(t, http.MethodPost, testURL, "10") - - // Wait for the producer/consumers to pick up the change - time.Sleep(1 * time.Second) - // Publish messages ensuring that they will go to all the partitions ctx := context.Background() for i := 0; i < 10; i++ { diff --git a/pulsaradmin/pkg/admin/topic.go b/pulsaradmin/pkg/admin/topic.go index c888827bf8..e6057413d2 100644 --- a/pulsaradmin/pkg/admin/topic.go +++ b/pulsaradmin/pkg/admin/topic.go @@ -75,6 +75,9 @@ type Topics interface { // All the rates are computed over a 1 minute window and are relative the last completed 1 minute period GetStats(utils.TopicName) (utils.TopicStats, error) + // GetStatsWithOption returns the stats for the topic + GetStatsWithOption(utils.TopicName, utils.GetStatsOptions) (utils.TopicStats, error) + // GetInternalStats returns the internal stats for the topic. GetInternalStats(utils.TopicName) (utils.PersistentTopicInternalStats, error) @@ -82,6 +85,9 @@ type Topics interface { // All the rates are computed over a 1 minute window and are relative the last completed 1 minute period GetPartitionedStats(utils.TopicName, bool) (utils.PartitionedTopicStats, error) + // GetPartitionedStatsWithOption returns the stats for the partitioned topic + GetPartitionedStatsWithOption(utils.TopicName, bool, utils.GetStatsOptions) (utils.PartitionedTopicStats, error) + // Terminate the topic and prevent any more messages being published on it Terminate(utils.TopicName) (utils.MessageID, error) @@ -395,6 +401,19 @@ func (t *topics) GetStats(topic utils.TopicName) (utils.TopicStats, error) { err := t.pulsar.Client.Get(endpoint, &stats) return stats, err } +func (t *topics) GetStatsWithOption(topic utils.TopicName, option utils.GetStatsOptions) (utils.TopicStats, error) { + var stats utils.TopicStats + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "stats") + params := map[string]string{ + "getPreciseBacklog": strconv.FormatBool(option.GetPreciseBacklog), + "subscriptionBacklogSize": strconv.FormatBool(option.SubscriptionBacklogSize), + "getEarliestTimeInBacklog": strconv.FormatBool(option.GetEarliestTimeInBacklog), + "excludePublishers": strconv.FormatBool(option.ExcludePublishers), + "excludeConsumers": strconv.FormatBool(option.ExcludeConsumers), + } + _, err := t.pulsar.Client.GetWithQueryParams(endpoint, &stats, params, true) + return stats, err +} func (t *topics) GetInternalStats(topic utils.TopicName) (utils.PersistentTopicInternalStats, error) { var stats utils.PersistentTopicInternalStats @@ -412,6 +431,21 @@ func (t *topics) GetPartitionedStats(topic utils.TopicName, perPartition bool) ( _, err := t.pulsar.Client.GetWithQueryParams(endpoint, &stats, params, true) return stats, err } +func (t *topics) GetPartitionedStatsWithOption(topic utils.TopicName, perPartition bool, + option utils.GetStatsOptions) (utils.PartitionedTopicStats, error) { + var stats utils.PartitionedTopicStats + endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "partitioned-stats") + params := map[string]string{ + "perPartition": strconv.FormatBool(perPartition), + "getPreciseBacklog": strconv.FormatBool(option.GetPreciseBacklog), + "subscriptionBacklogSize": strconv.FormatBool(option.SubscriptionBacklogSize), + "getEarliestTimeInBacklog": strconv.FormatBool(option.GetEarliestTimeInBacklog), + "excludePublishers": strconv.FormatBool(option.ExcludePublishers), + "excludeConsumers": strconv.FormatBool(option.ExcludeConsumers), + } + _, err := t.pulsar.Client.GetWithQueryParams(endpoint, &stats, params, true) + return stats, err +} func (t *topics) Terminate(topic utils.TopicName) (utils.MessageID, error) { endpoint := t.pulsar.endpoint(t.basePath, topic.GetRestPath(), "terminate") diff --git a/pulsaradmin/pkg/admin/topic_test.go b/pulsaradmin/pkg/admin/topic_test.go index 06c33f2ef5..a9b1f00283 100644 --- a/pulsaradmin/pkg/admin/topic_test.go +++ b/pulsaradmin/pkg/admin/topic_test.go @@ -18,12 +18,23 @@ package admin import ( + "context" + "fmt" + "log" "testing" + "time" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/stretchr/testify/assert" "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" ) +var ( + lookupURL = "pulsar://localhost:6650" +) + func TestCreateTopic(t *testing.T) { checkError := func(err error) { if err != nil { @@ -53,3 +64,152 @@ func TestCreateTopic(t *testing.T) { } t.Error("Couldn't find topic: " + topic) } + +func TestPartitionState(t *testing.T) { + randomName := newTopicName() + topic := "persistent://public/default/" + randomName + + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + + // Create partition topic + topicName, err := utils.GetTopicName(topic) + assert.NoError(t, err) + err = admin.Topics().Create(*topicName, 4) + assert.NoError(t, err) + + // Send message + ctx := context.Background() + + // create consumer + client, err := pulsar.NewClient(pulsar.ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + consumer, err := client.Subscribe(pulsar.ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + Type: pulsar.Exclusive, + }) + assert.Nil(t, err) + defer consumer.Close() + + // create producer + producer, err := client.CreateProducer(pulsar.ProducerOptions{ + Topic: topic, + DisableBatching: false, + }) + assert.Nil(t, err) + defer producer.Close() + + // send 10 messages + for i := 0; i < 10; i++ { + if _, err := producer.Send(ctx, &pulsar.ProducerMessage{ + Payload: []byte(fmt.Sprintf("hello-%d", i)), + Key: "pulsar", + Properties: map[string]string{ + "key-1": "pulsar-1", + }, + }); err != nil { + log.Fatal(err) + } + } + + stats, err := admin.Topics().GetPartitionedStatsWithOption(*topicName, true, utils.GetStatsOptions{ + GetPreciseBacklog: false, + SubscriptionBacklogSize: false, + GetEarliestTimeInBacklog: false, + ExcludePublishers: true, + ExcludeConsumers: true, + }) + assert.Nil(t, err) + assert.Equal(t, len(stats.Publishers), 0) + + for _, topicStats := range stats.Partitions { + assert.Equal(t, len(topicStats.Publishers), 0) + for _, subscriptionStats := range topicStats.Subscriptions { + assert.Equal(t, len(subscriptionStats.Consumers), 0) + } + } + + for _, subscriptionStats := range stats.Subscriptions { + assert.Equal(t, len(subscriptionStats.Consumers), 0) + } + +} +func TestNonPartitionState(t *testing.T) { + randomName := newTopicName() + topic := "persistent://public/default/" + randomName + + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + + // Create non-partition topic + topicName, err := utils.GetTopicName(topic) + assert.NoError(t, err) + err = admin.Topics().Create(*topicName, 0) + assert.NoError(t, err) + + // Send message + ctx := context.Background() + + // create consumer + client, err := pulsar.NewClient(pulsar.ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + consumer, err := client.Subscribe(pulsar.ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + Type: pulsar.Exclusive, + }) + assert.Nil(t, err) + defer consumer.Close() + + // create producer + producer, err := client.CreateProducer(pulsar.ProducerOptions{ + Topic: topic, + DisableBatching: false, + }) + assert.Nil(t, err) + defer producer.Close() + + // send 10 messages + for i := 0; i < 10; i++ { + if _, err := producer.Send(ctx, &pulsar.ProducerMessage{ + Payload: []byte(fmt.Sprintf("hello-%d", i)), + Key: "pulsar", + Properties: map[string]string{ + "key-1": "pulsar-1", + }, + }); err != nil { + log.Fatal(err) + } + } + + stats, err := admin.Topics().GetStatsWithOption(*topicName, utils.GetStatsOptions{ + GetPreciseBacklog: false, + SubscriptionBacklogSize: false, + GetEarliestTimeInBacklog: false, + ExcludePublishers: true, + ExcludeConsumers: true, + }) + assert.Nil(t, err) + assert.Equal(t, len(stats.Publishers), 0) + for _, subscriptionStats := range stats.Subscriptions { + assert.Equal(t, len(subscriptionStats.Consumers), 0) + } + +} + +func newTopicName() string { + return fmt.Sprintf("my-topic-%v", time.Now().Nanosecond()) +} diff --git a/pulsaradmin/pkg/utils/data.go b/pulsaradmin/pkg/utils/data.go index 55888aab2b..cc797d1892 100644 --- a/pulsaradmin/pkg/utils/data.go +++ b/pulsaradmin/pkg/utils/data.go @@ -465,3 +465,11 @@ type CompactedLedger struct { Offloaded bool `json:"offloaded"` UnderReplicated bool `json:"underReplicated"` } + +type GetStatsOptions struct { + GetPreciseBacklog bool `json:"get_precise_backlog"` + SubscriptionBacklogSize bool `json:"subscription_backlog_size"` + GetEarliestTimeInBacklog bool `json:"get_earliest_time_in_backlog"` + ExcludePublishers bool `json:"exclude_publishers"` + ExcludeConsumers bool `json:"exclude_consumers"` +} From 6d9cbd8ba3aa08b5285cceb0a4688dad57ae3171 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 6 Mar 2024 19:20:55 +0800 Subject: [PATCH 326/348] Remove `VERSION` and `stable.txt` files (#1158) ### Motivation We currently have two hardcoded version files in place: `VERSION` which represents the current version and `stable.txt` which represents the current stable version. However, I'm curious about the intended function of these files. Are they merely for developers to conveniently reference the current and stable versions? And what is our approach to managing these files? For instance, how do we determine what constitutes a stable version? If there's no compelling reason to keep these files, I'd propose to remove them. Please see more discussion here: https://lists.apache.org/thread/mrntb1y7ws173ohtdvz1v8q86pg50cn2 ### Modifications - Remove `VERSION` and `stable.txt` files. --- VERSION | 3 --- docs/release-process.md | 4 +--- stable.txt | 3 --- 3 files changed, 1 insertion(+), 9 deletions(-) delete mode 100644 VERSION delete mode 100644 stable.txt diff --git a/VERSION b/VERSION deleted file mode 100644 index 725659e7c0..0000000000 --- a/VERSION +++ /dev/null @@ -1,3 +0,0 @@ -// This version number refers to the currently released version number -// Please fix the version when release. -v0.12.0 diff --git a/docs/release-process.md b/docs/release-process.md index 0832f17a10..ab99ed675d 100644 --- a/docs/release-process.md +++ b/docs/release-process.md @@ -35,9 +35,7 @@ cd pulsar-client-go git checkout -b branch-0.X.0 origin/master ``` -2. Update the version and tag of a package. - -Update the information of the new release to the `VERSION` file and `stable.txt` file and send a PR for requesting the changes. +2. Create a tag for the release candidate. During the release process, you can create a "candidate" tag which will get promoted to the "real" final tag after verification and approval. diff --git a/stable.txt b/stable.txt deleted file mode 100644 index 8a5f2e8dfe..0000000000 --- a/stable.txt +++ /dev/null @@ -1,3 +0,0 @@ -// This version number refers to the current stable version, generally is `VERSION - 1`. -// Please fix the version when release. -v0.12.0 \ No newline at end of file From 143fa23228621414784cefae347286e797e4ef84 Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Fri, 8 Mar 2024 23:52:51 +0800 Subject: [PATCH 327/348] [Improve] getMessagesById gets all messages (#1194) ### Motivation To keep consistent with the Java client. Releted PR: /~https://github.com/apache/pulsar/pull/21918 ### Modifications Add `getMessagesById` interface. --------- Co-authored-by: Zike Yang --- pulsaradmin/pkg/admin/subscription.go | 23 +++- pulsaradmin/pkg/admin/subscription_test.go | 130 +++++++++++++++++++++ 2 files changed, 147 insertions(+), 6 deletions(-) create mode 100644 pulsaradmin/pkg/admin/subscription_test.go diff --git a/pulsaradmin/pkg/admin/subscription.go b/pulsaradmin/pkg/admin/subscription.go index 456de46cdb..8ddb584523 100644 --- a/pulsaradmin/pkg/admin/subscription.go +++ b/pulsaradmin/pkg/admin/subscription.go @@ -72,8 +72,11 @@ type Subscriptions interface { // PeekMessages peeks messages from a topic subscription PeekMessages(utils.TopicName, string, int) ([]*utils.Message, error) - // GetMessageByID gets message by its ledgerID and entryID + // Deprecated: Use GetMessagesByID() instead GetMessageByID(topic utils.TopicName, ledgerID, entryID int64) (*utils.Message, error) + + // GetMessagesByID gets messages by its ledgerID and entryID + GetMessagesByID(topic utils.TopicName, ledgerID, entryID int64) ([]*utils.Message, error) } type subscriptions struct { @@ -187,6 +190,18 @@ func (s *subscriptions) peekNthMessage(topic utils.TopicName, sName string, pos } func (s *subscriptions) GetMessageByID(topic utils.TopicName, ledgerID, entryID int64) (*utils.Message, error) { + messages, err := s.GetMessagesByID(topic, ledgerID, entryID) + if err != nil { + return nil, err + } + + if len(messages) == 0 { + return nil, nil + } + return messages[0], nil +} + +func (s *subscriptions) GetMessagesByID(topic utils.TopicName, ledgerID, entryID int64) ([]*utils.Message, error) { ledgerIDStr := strconv.FormatInt(ledgerID, 10) entryIDStr := strconv.FormatInt(entryID, 10) @@ -201,11 +216,7 @@ func (s *subscriptions) GetMessageByID(topic utils.TopicName, ledgerID, entryID if err != nil { return nil, err } - - if len(messages) == 0 { - return nil, nil - } - return messages[0], nil + return messages, nil } // safeRespClose is used to close a response body diff --git a/pulsaradmin/pkg/admin/subscription_test.go b/pulsaradmin/pkg/admin/subscription_test.go new file mode 100644 index 0000000000..c4ba717d37 --- /dev/null +++ b/pulsaradmin/pkg/admin/subscription_test.go @@ -0,0 +1,130 @@ +// 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 admin + +import ( + "context" + "fmt" + "sync" + "testing" + "time" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" + "github.com/stretchr/testify/assert" +) + +func TestGetMessagesByID(t *testing.T) { + + randomName := newTopicName() + topic := "persistent://public/default/" + randomName + + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + + ctx := context.Background() + + client, err := pulsar.NewClient(pulsar.ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + + // create producer + numberMessages := 10 + batchingMaxMessages := numberMessages / 2 + producer, err := client.CreateProducer(pulsar.ProducerOptions{ + Topic: topic, + DisableBatching: false, + BatchingMaxMessages: uint(batchingMaxMessages), + BatchingMaxPublishDelay: time.Second * 60, + }) + assert.Nil(t, err) + defer producer.Close() + + var wg sync.WaitGroup + wg.Add(numberMessages) + messageIDMap := make(map[string]int32) + for i := 0; i <= numberMessages; i++ { + producer.SendAsync(ctx, &pulsar.ProducerMessage{ + Payload: []byte(fmt.Sprintf("hello-%d", i)), + }, func(id pulsar.MessageID, message *pulsar.ProducerMessage, err error) { + assert.Nil(t, err) + messageIDMap[id.String()]++ + wg.Done() + }) + } + wg.Wait() + topicName, err := utils.GetTopicName(topic) + assert.NoError(t, err) + for id, i := range messageIDMap { + assert.Equal(t, i, int32(batchingMaxMessages)) + messageID, err := utils.ParseMessageID(id) + assert.Nil(t, err) + messages, err := admin.Subscriptions().GetMessagesByID(*topicName, messageID.LedgerID, messageID.EntryID) + assert.Nil(t, err) + assert.Equal(t, batchingMaxMessages, len(messages)) + } + + _, err = admin.Subscriptions().GetMessagesByID(*topicName, 1024, 2048) + assert.Errorf(t, err, "Message id not found") + +} + +func TestGetMessageByID(t *testing.T) { + randomName := newTopicName() + topic := "persistent://public/default/" + randomName + + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + + ctx := context.Background() + + client, err := pulsar.NewClient(pulsar.ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + + // create producer + producer, err := client.CreateProducer(pulsar.ProducerOptions{ + Topic: topic, + DisableBatching: false, + }) + assert.Nil(t, err) + defer producer.Close() + + messageID, err := producer.Send(ctx, &pulsar.ProducerMessage{ + Payload: []byte("hello"), + }) + assert.Nil(t, err) + + topicName, err := utils.GetTopicName(topic) + assert.NoError(t, err) + id, err := admin.Subscriptions().GetMessageByID(*topicName, messageID.LedgerID(), messageID.EntryID()) + assert.Nil(t, err) + assert.Equal(t, id.MessageID.LedgerID, messageID.LedgerID()) + assert.Equal(t, id.MessageID.EntryID, messageID.EntryID()) +} From 4f213796979cdf6ffb3477f595b25c4544d76811 Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Fri, 8 Mar 2024 23:53:30 +0800 Subject: [PATCH 328/348] [Improve]Change base image to apachepulsar/pulsar (#1195) ### Motivation There are related discussion records [here](/~https://github.com/apache/pulsar-client-go/pull/1037). It is recommended to switch the base image to `apachepulsar/pulsar`. ### Modifications Change base image to `apachepulsar/pulsar`. --- .github/workflows/ci.yml | 2 +- Dockerfile | 13 ++++++++----- Makefile | 3 +-- scripts/run-ci.sh | 3 ++- 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 9ad0b3f255..7b7700515f 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -38,7 +38,7 @@ jobs: runs-on: ubuntu-latest strategy: matrix: - go-version: ['1.18', '1.19', '1.20', '1.21'] + go-version: ['1.18', '1.19', '1.20', '1.21.0'] steps: - uses: actions/checkout@v3 - name: clean docker cache diff --git a/Dockerfile b/Dockerfile index 1c137fc096..62d22bf06d 100644 --- a/Dockerfile +++ b/Dockerfile @@ -18,14 +18,17 @@ # Explicit version of Pulsar and Golang images should be # set via the Makefile or CLI ARG PULSAR_IMAGE=apachepulsar/pulsar:latest -ARG GOLANG_IMAGE=golang:latest +FROM $PULSAR_IMAGE +USER root +ARG GO_VERSION=1.18 -FROM $PULSAR_IMAGE as pulsar -FROM $GOLANG_IMAGE +RUN curl -L https://dl.google.com/go/go${GO_VERSION}.linux-amd64.tar.gz -o golang.tar.gz && \ + mkdir -p /pulsar/go && tar -C /pulsar -xzf golang.tar.gz -RUN apt-get update && apt-get install -y openjdk-17-jre ca-certificates +ENV PATH /pulsar/go/bin:$PATH + +RUN apt-get update && apt-get install -y git && apt-get install -y gcc -COPY --from=pulsar /pulsar /pulsar ### Add pulsar config COPY integration-tests/certs /pulsar/certs diff --git a/Makefile b/Makefile index d044237221..527d53acf2 100644 --- a/Makefile +++ b/Makefile @@ -21,7 +21,6 @@ IMAGE_NAME = pulsar-client-go-test:latest PULSAR_VERSION ?= 3.2.0 PULSAR_IMAGE = apachepulsar/pulsar:$(PULSAR_VERSION) GO_VERSION ?= 1.18 -GOLANG_IMAGE = golang:$(GO_VERSION) # Golang standard bin directory. GOPATH ?= $(shell go env GOPATH) @@ -39,7 +38,7 @@ bin/golangci-lint: GOBIN=$(shell pwd)/bin go install github.com/golangci/golangci-lint/cmd/golangci-lint@v1.51.2 container: - docker build -t ${IMAGE_NAME} --build-arg GOLANG_IMAGE="${GOLANG_IMAGE}" \ + docker build -t ${IMAGE_NAME} --build-arg GO_VERSION="${GO_VERSION}" \ --build-arg PULSAR_IMAGE="${PULSAR_IMAGE}" . test: container diff --git a/scripts/run-ci.sh b/scripts/run-ci.sh index cc4f6a1e74..83246a39f0 100755 --- a/scripts/run-ci.sh +++ b/scripts/run-ci.sh @@ -19,7 +19,8 @@ set -e -x -export GOPATH=/ +export GOPATH=/pulsar/go +export GOCACHE=/tmp/go-cache # Install dependencies go mod download From 3935fd7b1efe0afe7643fdcc5687559a4d00a40b Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Sat, 9 Mar 2024 13:33:30 +0800 Subject: [PATCH 329/348] Add change log for 0.12.1 (#1189) ### Motivation Add 0.12.1 change log --- CHANGELOG.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7a4c81cb74..f25de20997 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,6 +23,15 @@ All notable changes to this project will be documented in this file. +[0.12.1] 2024-02-29 + +- [fix] Fix Infinite Loop in Reader's `HasNext` Function by @RobertIndie in [#1182](/~https://github.com/apache/pulsar-client-go/pull/1182) +- [fix] Fix available permits in MessageReceived by @panszobe in [#1181](/~https://github.com/apache/pulsar-client-go/pull/1181) +- [feat] Support partitioned topic reader by @RobertIndie in [#1178](/~https://github.com/apache/pulsar-client-go/pull/1178) +- [fix] Fix BytesSchema by @petermnhull in [#1173](/~https://github.com/apache/pulsar-client-go/pull/1173) +- [fix] Respect context cancellation in Flush by @jayshrivastava in [#1165](/~https://github.com/apache/pulsar-client-go/pull/1165) +- [fix] Fix SIGSEGV with zstd compression enabled by @RobertIndie in [#1164](/~https://github.com/apache/pulsar-client-go/pull/1164) + [0.12.0] 2024-01-10 ## What's Changed From 3693b3695e2f072f9506b64b5e3000e5f107070d Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Thu, 21 Mar 2024 15:39:08 +0800 Subject: [PATCH 330/348] [fix] Change the wrong `SourceInstanceStatusData` in SinkInstanceStatus (#1199) --- pulsaradmin/pkg/utils/sink_status.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsaradmin/pkg/utils/sink_status.go b/pulsaradmin/pkg/utils/sink_status.go index 6cdb091fac..a2651959ce 100644 --- a/pulsaradmin/pkg/utils/sink_status.go +++ b/pulsaradmin/pkg/utils/sink_status.go @@ -28,8 +28,8 @@ type SinkStatus struct { } type SinkInstanceStatus struct { - InstanceID int `json:"instanceId"` - Status SourceInstanceStatusData `json:"status"` + InstanceID int `json:"instanceId"` + Status SinkInstanceStatusData `json:"status"` } type SinkInstanceStatusData struct { From 393f80b4b93faa36936380b643426026a2b2cd02 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 21 Mar 2024 22:24:44 +0800 Subject: [PATCH 331/348] chore(deps): bump google.golang.org/protobuf from 1.30.0 to 1.33.0 (#1198) Bumps google.golang.org/protobuf from 1.30.0 to 1.33.0. --- updated-dependencies: - dependency-name: google.golang.org/protobuf dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 14 ++------------ 2 files changed, 3 insertions(+), 13 deletions(-) diff --git a/go.mod b/go.mod index 3257d51047..511648711b 100644 --- a/go.mod +++ b/go.mod @@ -27,7 +27,7 @@ require ( golang.org/x/mod v0.8.0 golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 golang.org/x/time v0.0.0-20191024005414-555d28b269f0 - google.golang.org/protobuf v1.30.0 + google.golang.org/protobuf v1.33.0 ) require ( diff --git a/go.sum b/go.sum index 50a1ba3ebb..0a768a0eff 100644 --- a/go.sum +++ b/go.sum @@ -321,8 +321,6 @@ golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzB golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.5.1 h1:OJxoQ/rynoF0dcCdI7cLPktw/hR2cueqYfjm43oqK38= -golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/mod v0.8.0 h1:LUYupSeNrTNCGzR/hVBk2NHZO4hXcVaW1k4Qx7rjPx8= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -357,8 +355,6 @@ golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220225172249-27dd8689420f h1:oA4XRj0qtSt8Yo1Zms0CUlsT3KG69V2UGQWPBxujDmc= -golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -420,13 +416,9 @@ golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210819135213-f52c844e1c1c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220204135822-1c1b9b1eba6a h1:ppl5mZgokTT8uPkmYOyEUmPTr3ypaKkg5eFOGrAmxxE= -golang.org/x/sys v0.0.0-20220204135822-1c1b9b1eba6a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 h1:JGgROgKl9N8DuW20oFS5gxc+lE67/N3FcwmBPMe7ArY= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.13.0 h1:bb+I9cTfFazGW51MZqBVmZy7+JEJMouUHTUSKVQLBek= golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -435,8 +427,6 @@ golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3 golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= -golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.13.0 h1:ablQoSUd0tRdKxZewP80B+BaqeKJuVhuRxj/dkrun3k= golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -566,8 +556,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= -google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= +google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= From 3096ba59d93570dc48417da36042e499a670030b Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Wed, 10 Apr 2024 00:53:11 +0800 Subject: [PATCH 332/348] [Improve] Add admin api HealthCheckWithTopicVersion (#1200) * add HealthCheckWithTopicVersion api * add HealthCheckWithTopicVersion api * add HealthCheckWithTopicVersion api * read from admin-token * fix with cr --- integration-tests/tokens/admin-token | 1 + pulsaradmin/pkg/admin/brokers.go | 12 ++++++-- pulsaradmin/pkg/admin/brokers_test.go | 44 +++++++++++++++++++++++++++ pulsaradmin/pkg/utils/data.go | 11 +++++++ 4 files changed, 66 insertions(+), 2 deletions(-) create mode 100644 integration-tests/tokens/admin-token create mode 100644 pulsaradmin/pkg/admin/brokers_test.go diff --git a/integration-tests/tokens/admin-token b/integration-tests/tokens/admin-token new file mode 100644 index 0000000000..67acef15fd --- /dev/null +++ b/integration-tests/tokens/admin-token @@ -0,0 +1 @@ +eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJhZG1pbiJ9.MKSR5Mb2wu_FQlMYACv2i4ubMCn4h4Dj_aIDo1dPsDk \ No newline at end of file diff --git a/pulsaradmin/pkg/admin/brokers.go b/pulsaradmin/pkg/admin/brokers.go index 79fcb092ef..e178610c30 100644 --- a/pulsaradmin/pkg/admin/brokers.go +++ b/pulsaradmin/pkg/admin/brokers.go @@ -53,8 +53,11 @@ type Brokers interface { // GetAllDynamicConfigurations returns values of all overridden dynamic-configs GetAllDynamicConfigurations() (map[string]string, error) - // HealthCheck run a health check on the broker + // Deprecated: Use HealthCheckWithTopicVersion instead HealthCheck() error + + // HealthCheckWithTopicVersion run a health check on the broker + HealthCheckWithTopicVersion(utils.TopicVersion) error } type broker struct { @@ -142,9 +145,14 @@ func (b *broker) GetAllDynamicConfigurations() (map[string]string, error) { } func (b *broker) HealthCheck() error { + return b.HealthCheckWithTopicVersion(utils.TopicVersionV1) +} +func (b *broker) HealthCheckWithTopicVersion(topicVersion utils.TopicVersion) error { endpoint := b.pulsar.endpoint(b.basePath, "/health") - buf, err := b.pulsar.Client.GetWithQueryParams(endpoint, nil, nil, false) + buf, err := b.pulsar.Client.GetWithQueryParams(endpoint, nil, map[string]string{ + "topicVersion": topicVersion.String(), + }, false) if err != nil { return err } diff --git a/pulsaradmin/pkg/admin/brokers_test.go b/pulsaradmin/pkg/admin/brokers_test.go new file mode 100644 index 0000000000..d48ce7cbd4 --- /dev/null +++ b/pulsaradmin/pkg/admin/brokers_test.go @@ -0,0 +1,44 @@ +// 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 admin + +import ( + "os" + "testing" + + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" + "github.com/stretchr/testify/assert" +) + +func TestBrokerHealthCheckWithTopicVersion(t *testing.T) { + readFile, err := os.ReadFile("../../../integration-tests/tokens/admin-token") + assert.NoError(t, err) + cfg := &config.Config{ + Token: string(readFile), + } + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + err = admin.Brokers().HealthCheck() + assert.NoError(t, err) + err = admin.Brokers().HealthCheckWithTopicVersion(utils.TopicVersionV1) + assert.NoError(t, err) + err = admin.Brokers().HealthCheckWithTopicVersion(utils.TopicVersionV2) + assert.NoError(t, err) +} diff --git a/pulsaradmin/pkg/utils/data.go b/pulsaradmin/pkg/utils/data.go index cc797d1892..4363dd349e 100644 --- a/pulsaradmin/pkg/utils/data.go +++ b/pulsaradmin/pkg/utils/data.go @@ -473,3 +473,14 @@ type GetStatsOptions struct { ExcludePublishers bool `json:"exclude_publishers"` ExcludeConsumers bool `json:"exclude_consumers"` } + +type TopicVersion string + +const ( + TopicVersionV1 TopicVersion = "V1" + TopicVersionV2 TopicVersion = "V2" +) + +func (t TopicVersion) String() string { + return string(t) +} From 10819739d8c0f66bf97745ba0bdf5630e09f58d4 Mon Sep 17 00:00:00 2001 From: zhou zhuohan <843520313@qq.com> Date: Fri, 12 Apr 2024 13:08:54 +0800 Subject: [PATCH 333/348] [improve] Update topic admin interface comment, add topic admin test cases (#1202) --- integration-tests/conf/standalone.conf | 3 + pulsaradmin/pkg/admin/topic.go | 179 ++++++++++++++++--- pulsaradmin/pkg/admin/topic_test.go | 227 +++++++++++++++++++++++++ 3 files changed, 384 insertions(+), 25 deletions(-) diff --git a/integration-tests/conf/standalone.conf b/integration-tests/conf/standalone.conf index c816c8fd16..ccb91f376a 100644 --- a/integration-tests/conf/standalone.conf +++ b/integration-tests/conf/standalone.conf @@ -83,6 +83,9 @@ maxUnackedMessagesPerConsumer=50000 # Set maxMessageSize to 1MB rather than the default value 5MB for testing maxMessageSize=1048576 +# enable topic level policies to test topic admin functions +topicLevelPoliciesEnabled=true + ### --- Authentication --- ### # Enable TLS diff --git a/pulsaradmin/pkg/admin/topic.go b/pulsaradmin/pkg/admin/topic.go index e6057413d2..7badc63400 100644 --- a/pulsaradmin/pkg/admin/topic.go +++ b/pulsaradmin/pkg/admin/topic.go @@ -26,16 +26,35 @@ import ( // Topics is admin interface for topics management type Topics interface { - // Create a topic - Create(utils.TopicName, int) error - - // Delete a topic - Delete(utils.TopicName, bool, bool) error + // Create a partitioned or non-partitioned topic + // + // @param topic + // topicName struct + // @param partitions + // number of topic partitions, + // when setting to 0, it will create a non-partitioned topic + Create(topic utils.TopicName, partitions int) error + + // Delete a topic, this function can delete both partitioned or non-partitioned topic + // + // @param topic + // topicName struct + // @param force + // delete topic forcefully + // @param nonPartitioned + // when set to true, topic will be treated as a non-partitioned topic + // Otherwise it will be treated as a partitioned topic + Delete(topic utils.TopicName, force bool, nonPartitioned bool) error // Update number of partitions of a non-global partitioned topic // It requires partitioned-topic to be already exist and number of new partitions must be greater than existing // number of partitions. Decrementing number of partitions requires deletion of topic which is not supported. - Update(utils.TopicName, int) error + // + // @param topic + // topicName struct + // @param partitions + // number of new partitions of already exist partitioned-topic + Update(topic utils.TopicName, partitions int) error // GetMetadata returns metadata of a partitioned topic GetMetadata(utils.TopicName) (utils.PartitionedTopicMetadata, error) @@ -52,12 +71,24 @@ type Topics interface { GetPermissions(utils.TopicName) (map[string][]utils.AuthAction, error) // GrantPermission grants a new permission to a client role on a single topic - GrantPermission(utils.TopicName, string, []utils.AuthAction) error + // + // @param topic + // topicName struct + // @param role + // client role to which grant permission + // @param action + // auth actions (e.g. produce and consume) + GrantPermission(topic utils.TopicName, role string, action []utils.AuthAction) error // RevokePermission revokes permissions to a client role on a single topic. If the permission // was not set at the topic level, but rather at the namespace level, this operation will // return an error (HTTP status code 412). - RevokePermission(utils.TopicName, string) error + // + // @param topic + // topicName struct + // @param role + // client role to which remove permissions + RevokePermission(topic utils.TopicName, role string) error // Lookup a topic returns the broker URL that serves the topic Lookup(utils.TopicName) (utils.LookupData, error) @@ -69,24 +100,56 @@ type Topics interface { GetLastMessageID(utils.TopicName) (utils.MessageID, error) // GetMessageID returns the message Id by timestamp(ms) of a topic - GetMessageID(utils.TopicName, int64) (utils.MessageID, error) - - // GetStats returns the stats for the topic - // All the rates are computed over a 1 minute window and are relative the last completed 1 minute period + // + // @param topic + // topicName struct + // @param timestamp + // absolute timestamp (in ms) + GetMessageID(topic utils.TopicName, timestamp int64) (utils.MessageID, error) + + // GetStats returns the stats for the topic. + // + // All the rates are computed over a 1-minute window and are relative the last completed 1-minute period GetStats(utils.TopicName) (utils.TopicStats, error) // GetStatsWithOption returns the stats for the topic - GetStatsWithOption(utils.TopicName, utils.GetStatsOptions) (utils.TopicStats, error) + // + // All the rates are computed over a 1-minute window and are relative the last completed 1-minute period + // + // @param topic + // topicName struct + // @param option + // request option, e.g. get_precise_backlog or subscription_backlog_size + GetStatsWithOption(topic utils.TopicName, option utils.GetStatsOptions) (utils.TopicStats, error) // GetInternalStats returns the internal stats for the topic. GetInternalStats(utils.TopicName) (utils.PersistentTopicInternalStats, error) // GetPartitionedStats returns the stats for the partitioned topic - // All the rates are computed over a 1 minute window and are relative the last completed 1 minute period - GetPartitionedStats(utils.TopicName, bool) (utils.PartitionedTopicStats, error) + // + // All the rates are computed over a 1-minute window and are relative the last completed 1-minute period + // + // @param topic + // topicName struct + // @param perPartition + // flag to get stats per partition + GetPartitionedStats(topic utils.TopicName, perPartition bool) (utils.PartitionedTopicStats, error) // GetPartitionedStatsWithOption returns the stats for the partitioned topic - GetPartitionedStatsWithOption(utils.TopicName, bool, utils.GetStatsOptions) (utils.PartitionedTopicStats, error) + // + // All the rates are computed over a 1-minute window and are relative the last completed 1-minute period + // + // @param topic + // topicName struct + // @param perPartition + // flag to get stats per partition + // @param option + // request option, e.g. get_precise_backlog or subscription_backlog_size + GetPartitionedStatsWithOption( + topic utils.TopicName, + perPartition bool, + option utils.GetStatsOptions, + ) (utils.PartitionedTopicStats, error) // Terminate the topic and prevent any more messages being published on it Terminate(utils.TopicName) (utils.MessageID, error) @@ -111,7 +174,12 @@ type Topics interface { GetMessageTTL(utils.TopicName) (int, error) // SetMessageTTL Set the message TTL for a topic - SetMessageTTL(utils.TopicName, int) error + // + // @param topic + // topicName struct + // @param messageTTL + // Message TTL in second + SetMessageTTL(topic utils.TopicName, messageTTL int) error // RemoveMessageTTL Remove the message TTL for a topic RemoveMessageTTL(utils.TopicName) error @@ -120,7 +188,12 @@ type Topics interface { GetMaxProducers(utils.TopicName) (int, error) // SetMaxProducers Set max number of producers for a topic - SetMaxProducers(utils.TopicName, int) error + // + // @param topic + // topicName struct + // @param maxProducers + // max number of producer + SetMaxProducers(topic utils.TopicName, maxProducers int) error // RemoveMaxProducers Remove max number of producers for a topic RemoveMaxProducers(utils.TopicName) error @@ -129,7 +202,12 @@ type Topics interface { GetMaxConsumers(utils.TopicName) (int, error) // SetMaxConsumers Set max number of consumers for a topic - SetMaxConsumers(utils.TopicName, int) error + // + // @param topic + // topicName struct + // @param maxConsumers + // max number of consumer + SetMaxConsumers(topic utils.TopicName, maxConsumers int) error // RemoveMaxConsumers Remove max number of consumers for a topic RemoveMaxConsumers(utils.TopicName) error @@ -138,7 +216,12 @@ type Topics interface { GetMaxUnackMessagesPerConsumer(utils.TopicName) (int, error) // SetMaxUnackMessagesPerConsumer Set max unacked messages policy on consumer for a topic - SetMaxUnackMessagesPerConsumer(utils.TopicName, int) error + // + // @param topic + // topicName struct + // @param maxUnackedNum + // max unAcked messages on each consumer + SetMaxUnackMessagesPerConsumer(topic utils.TopicName, maxUnackedNum int) error // RemoveMaxUnackMessagesPerConsumer Remove max unacked messages policy on consumer for a topic RemoveMaxUnackMessagesPerConsumer(utils.TopicName) error @@ -147,7 +230,12 @@ type Topics interface { GetMaxUnackMessagesPerSubscription(utils.TopicName) (int, error) // SetMaxUnackMessagesPerSubscription Set max unacked messages policy on subscription for a topic - SetMaxUnackMessagesPerSubscription(utils.TopicName, int) error + // + // @param topic + // topicName struct + // @param maxUnackedNum + // max unAcked messages on subscription of a topic + SetMaxUnackMessagesPerSubscription(topic utils.TopicName, maxUnackedNum int) error // RemoveMaxUnackMessagesPerSubscription Remove max unacked messages policy on subscription for a topic RemoveMaxUnackMessagesPerSubscription(utils.TopicName) error @@ -192,13 +280,24 @@ type Topics interface { GetDeduplicationStatus(utils.TopicName) (bool, error) // SetDeduplicationStatus Set the deduplication policy for a topic - SetDeduplicationStatus(utils.TopicName, bool) error + // + // @param topic + // topicName struct + // @param enabled + // set enable or disable deduplication of the topic + SetDeduplicationStatus(topic utils.TopicName, enabled bool) error // RemoveDeduplicationStatus Remove the deduplication policy for a topic RemoveDeduplicationStatus(utils.TopicName) error // GetRetention returns the retention configuration for a topic - GetRetention(utils.TopicName, bool) (*utils.RetentionPolicies, error) + // + // @param topic + // topicName struct + // @param applied + // when set to true, function will try to find policy applied to this topic + // in namespace or broker level, if no policy set in topic level + GetRetention(topic utils.TopicName, applied bool) (*utils.RetentionPolicies, error) // RemoveRetention removes the retention configuration on a topic RemoveRetention(utils.TopicName) error @@ -206,16 +305,35 @@ type Topics interface { // SetRetention sets the retention policy for a topic SetRetention(utils.TopicName, utils.RetentionPolicies) error - // Get the compaction threshold for a topic + // GetCompactionThreshold Get the compaction threshold for a topic. + // + // i.e. The maximum number of bytes can have before compaction is triggered. + // + // @param topic + // topicName struct + // @param applied + // when set to true, function will try to find policy applied to this topic + // in namespace or broker level, if no policy set in topic level GetCompactionThreshold(topic utils.TopicName, applied bool) (int64, error) - // Set the compaction threshold for a topic + // SetCompactionThreshold Set the compaction threshold for a topic + // + // @param topic + // topicName struct + // @param threshold + // maximum number of backlog bytes before compaction is triggered SetCompactionThreshold(topic utils.TopicName, threshold int64) error // Remove compaction threshold for a topic RemoveCompactionThreshold(utils.TopicName) error // GetBacklogQuotaMap returns backlog quota map for a topic + // + // @param topic + // topicName struct + // @param applied + // when set to true, function will try to find policy applied to this topic + // in namespace or broker level, if no policy set in topic level GetBacklogQuotaMap(topic utils.TopicName, applied bool) (map[utils.BacklogQuotaType]utils.BacklogQuota, error) // SetBacklogQuota sets a backlog quota for a topic @@ -225,6 +343,12 @@ type Topics interface { RemoveBacklogQuota(utils.TopicName, utils.BacklogQuotaType) error // GetInactiveTopicPolicies gets the inactive topic policies on a topic + // + // @param topic + // topicName struct + // @param applied + // when set to true, function will try to find policy applied to this topic + // in namespace or broker level, if no policy set in topic level GetInactiveTopicPolicies(topic utils.TopicName, applied bool) (utils.InactiveTopicPolicies, error) // RemoveInactiveTopicPolicies removes inactive topic policies from a topic @@ -237,6 +361,11 @@ type Topics interface { GetReplicationClusters(topic utils.TopicName) ([]string, error) // SetReplicationClusters sets the replication clusters on a topic + // + // @param topic + // topicName struct + // @param data + // list of replication cluster id SetReplicationClusters(topic utils.TopicName, data []string) error } diff --git a/pulsaradmin/pkg/admin/topic_test.go b/pulsaradmin/pkg/admin/topic_test.go index a9b1f00283..a3609ff64e 100644 --- a/pulsaradmin/pkg/admin/topic_test.go +++ b/pulsaradmin/pkg/admin/topic_test.go @@ -213,3 +213,230 @@ func TestNonPartitionState(t *testing.T) { func newTopicName() string { return fmt.Sprintf("my-topic-%v", time.Now().Nanosecond()) } + +func TestDeleteNonPartitionedTopic(t *testing.T) { + randomName := newTopicName() + topic := "persistent://public/default/" + randomName + + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + topicName, err := utils.GetTopicName(topic) + assert.NoError(t, err) + err = admin.Topics().Create(*topicName, 0) + assert.NoError(t, err) + err = admin.Topics().Delete(*topicName, false, true) + assert.NoError(t, err) + topicList, err := admin.Namespaces().GetTopics("public/default") + assert.NoError(t, err) + isTopicExist := false + for _, topicIterator := range topicList { + if topicIterator == topic { + isTopicExist = true + } + } + assert.Equal(t, false, isTopicExist) +} + +func TestDeletePartitionedTopic(t *testing.T) { + randomName := newTopicName() + topic := "persistent://public/default/" + randomName + + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + topicName, err := utils.GetTopicName(topic) + assert.NoError(t, err) + err = admin.Topics().Create(*topicName, 3) + assert.NoError(t, err) + err = admin.Topics().Delete(*topicName, false, false) + assert.NoError(t, err) + topicList, err := admin.Namespaces().GetTopics("public/default") + assert.NoError(t, err) + isTopicExist := false + for _, topicIterator := range topicList { + if topicIterator == topic { + isTopicExist = true + } + } + assert.Equal(t, false, isTopicExist) +} + +func TestUpdateTopicPartitions(t *testing.T) { + randomName := newTopicName() + topic := "persistent://public/default/" + randomName + + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + topicName, err := utils.GetTopicName(topic) + assert.NoError(t, err) + err = admin.Topics().Create(*topicName, 3) + assert.NoError(t, err) + topicMetadata, err := admin.Topics().GetMetadata(*topicName) + assert.NoError(t, err) + assert.Equal(t, 3, topicMetadata.Partitions) + + err = admin.Topics().Update(*topicName, 4) + assert.NoError(t, err) + topicMetadata, err = admin.Topics().GetMetadata(*topicName) + assert.NoError(t, err) + assert.Equal(t, 4, topicMetadata.Partitions) +} + +func TestGetMessageID(t *testing.T) { + randomName := newTopicName() + topic := "persistent://public/default/" + randomName + topicPartitionZero := topic + "-partition-0" + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + topicName, err := utils.GetTopicName(topic) + assert.NoError(t, err) + topicPartitionZeroName, err := utils.GetTopicName(topicPartitionZero) + assert.NoError(t, err) + err = admin.Topics().Create(*topicName, 1) + assert.NoError(t, err) + ctx := context.Background() + + // create consumer + client, err := pulsar.NewClient(pulsar.ClientOptions{ + URL: lookupURL, + }) + assert.NoError(t, err) + defer client.Close() + consumer, err := client.Subscribe(pulsar.ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + Type: pulsar.Exclusive, + }) + assert.NoError(t, err) + defer consumer.Close() + + // create producer + producer, err := client.CreateProducer(pulsar.ProducerOptions{ + Topic: topic, + DisableBatching: false, + }) + assert.NoError(t, err) + defer producer.Close() + _, err = producer.Send(ctx, &pulsar.ProducerMessage{ + Payload: []byte("hello"), + Key: "pulsar", + Properties: map[string]string{ + "key-1": "pulsar-1", + }, + }) + assert.NoError(t, err) + + // ack message + msg, err := consumer.Receive(ctx) + assert.NoError(t, err) + assert.Equal(t, []byte("hello"), msg.Payload()) + assert.Equal(t, "pulsar", msg.Key()) + err = consumer.Ack(msg) + assert.NoError(t, err) + + messageID, err := admin.Topics().GetMessageID( + *topicPartitionZeroName, + msg.PublishTime().Unix()*1000-1000, + ) + assert.NoError(t, err) + assert.Equal(t, msg.ID().EntryID(), messageID.EntryID) + assert.Equal(t, msg.ID().LedgerID(), messageID.LedgerID) + assert.Equal(t, int(msg.ID().PartitionIdx()), messageID.PartitionIndex) +} + +func TestMessageTTL(t *testing.T) { + randomName := newTopicName() + topic := "persistent://public/default/" + randomName + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + topicName, err := utils.GetTopicName(topic) + assert.NoError(t, err) + err = admin.Topics().Create(*topicName, 4) + assert.NoError(t, err) + + messageTTL, err := admin.Topics().GetMessageTTL(*topicName) + assert.NoError(t, err) + assert.Equal(t, 0, messageTTL) + err = admin.Topics().SetMessageTTL(*topicName, 600) + assert.NoError(t, err) + // topic policy is an async operation, + // so we need to wait for a while to get current value + assert.Eventually( + t, + func() bool { + messageTTL, err = admin.Topics().GetMessageTTL(*topicName) + return err == nil && messageTTL == 600 + }, + 10*time.Second, + 100*time.Millisecond, + ) + err = admin.Topics().RemoveMessageTTL(*topicName) + assert.NoError(t, err) + assert.Eventually( + t, + func() bool { + messageTTL, err = admin.Topics().GetMessageTTL(*topicName) + return err == nil && messageTTL == 0 + }, + 10*time.Second, + 100*time.Millisecond, + ) +} + +func TestRetention(t *testing.T) { + randomName := newTopicName() + topic := "persistent://public/default/" + randomName + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + topicName, err := utils.GetTopicName(topic) + assert.NoError(t, err) + err = admin.Topics().Create(*topicName, 4) + assert.NoError(t, err) + + topicRetentionPolicy, err := admin.Topics().GetRetention(*topicName, false) + assert.NoError(t, err) + assert.Equal(t, int64(0), topicRetentionPolicy.RetentionSizeInMB) + assert.Equal(t, 0, topicRetentionPolicy.RetentionTimeInMinutes) + err = admin.Topics().SetRetention(*topicName, utils.RetentionPolicies{ + RetentionSizeInMB: 20480, + RetentionTimeInMinutes: 1440, + }) + assert.NoError(t, err) + // topic policy is an async operation, + // so we need to wait for a while to get current value + assert.Eventually( + t, + func() bool { + topicRetentionPolicy, err = admin.Topics().GetRetention(*topicName, false) + return err == nil && + topicRetentionPolicy.RetentionSizeInMB == int64(20480) && + topicRetentionPolicy.RetentionTimeInMinutes == 1440 + }, + 10*time.Second, + 100*time.Millisecond, + ) + err = admin.Topics().RemoveRetention(*topicName) + assert.NoError(t, err) + assert.Eventually( + t, + func() bool { + topicRetentionPolicy, err = admin.Topics().GetRetention(*topicName, false) + return err == nil && + topicRetentionPolicy.RetentionSizeInMB == int64(0) && + topicRetentionPolicy.RetentionTimeInMinutes == 0 + }, + 10*time.Second, + 100*time.Millisecond, + ) +} From c3e94e243a730ae22d59bf9d330c4539733b7eef Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 16 Apr 2024 19:29:08 -0700 Subject: [PATCH 334/348] [fix][build] Build test container image using current hardware platform (#1205) ### Motivation Test container image is built for `amd64` platforms only. This makes it difficult to test and run on others, notably Apple Silicon chips running `arm64`. ### Modifications The source of the problem is using the hardcoded `amd64` Go binary distribution during the image build. On Apple Silicon, running `make test` yields the following error: ``` ... docker run -i pulsar-client-go-test:latest bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci.sh" + export GOPATH=/pulsar/go + GOPATH=/pulsar/go + export GOCACHE=/tmp/go-cache + GOCACHE=/tmp/go-cache + go mod download rosetta error: failed to open elf at /lib64/ld-linux-x86-64.so.2 ./scripts/run-ci.sh: line 26: 7 Trace/breakpoint trap go mod download make: *** [test] Error 133 ``` Sourcing the local platform name from `uname -m` and using it to download the corresponding Go distribution solves the issue: ``` ... docker run -i pulsar-client-go-test:latest bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci.sh" + export GOPATH=/pulsar/go + GOPATH=/pulsar/go + export GOCACHE=/tmp/go-cache + GOCACHE=/tmp/go-cache + go mod download + go build ./pulsar + go build -o bin/pulsar-perf ./perf + scripts/pulsar-test-service-start.sh ... ``` --- Dockerfile | 3 ++- Makefile | 7 +++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/Dockerfile b/Dockerfile index 62d22bf06d..818a106a9f 100644 --- a/Dockerfile +++ b/Dockerfile @@ -21,8 +21,9 @@ ARG PULSAR_IMAGE=apachepulsar/pulsar:latest FROM $PULSAR_IMAGE USER root ARG GO_VERSION=1.18 +ARG ARCH=amd64 -RUN curl -L https://dl.google.com/go/go${GO_VERSION}.linux-amd64.tar.gz -o golang.tar.gz && \ +RUN curl -L https://dl.google.com/go/go${GO_VERSION}.linux-${ARCH}.tar.gz -o golang.tar.gz && \ mkdir -p /pulsar/go && tar -C /pulsar -xzf golang.tar.gz ENV PATH /pulsar/go/bin:$PATH diff --git a/Makefile b/Makefile index 527d53acf2..62e44166cc 100644 --- a/Makefile +++ b/Makefile @@ -21,6 +21,7 @@ IMAGE_NAME = pulsar-client-go-test:latest PULSAR_VERSION ?= 3.2.0 PULSAR_IMAGE = apachepulsar/pulsar:$(PULSAR_VERSION) GO_VERSION ?= 1.18 +CONTAINER_ARCH ?= $(shell uname -m | sed s/x86_64/amd64/) # Golang standard bin directory. GOPATH ?= $(shell go env GOPATH) @@ -38,8 +39,10 @@ bin/golangci-lint: GOBIN=$(shell pwd)/bin go install github.com/golangci/golangci-lint/cmd/golangci-lint@v1.51.2 container: - docker build -t ${IMAGE_NAME} --build-arg GO_VERSION="${GO_VERSION}" \ - --build-arg PULSAR_IMAGE="${PULSAR_IMAGE}" . + docker build -t ${IMAGE_NAME} \ + --build-arg GO_VERSION="${GO_VERSION}" \ + --build-arg PULSAR_IMAGE="${PULSAR_IMAGE}" \ + --build-arg ARCH="${CONTAINER_ARCH}" . test: container docker run -i ${IMAGE_NAME} bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci.sh" From a6f4b712c130a2219a73bbfd7d78717bbc9c739f Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Thu, 18 Apr 2024 09:15:22 +0800 Subject: [PATCH 335/348] expose RuntimeFlags (#1204) --- pulsaradmin/pkg/utils/data.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsaradmin/pkg/utils/data.go b/pulsaradmin/pkg/utils/data.go index 4363dd349e..1e67e3c71c 100644 --- a/pulsaradmin/pkg/utils/data.go +++ b/pulsaradmin/pkg/utils/data.go @@ -93,6 +93,7 @@ type FunctionData struct { Secrets string `json:"secretsString"` DestinationFile string `json:"destinationFile"` Path string `json:"path"` + RuntimeFlags string `json:"runtimeFlags,omitempty"` FuncConf *FunctionConfig `json:"-"` } @@ -138,7 +139,8 @@ type SourceData struct { SourceConf *SourceConfig `json:"-,omitempty"` InstanceID string `json:"instanceId,omitempty"` - UpdateAuthData bool `json:"updateAuthData,omitempty"` + UpdateAuthData bool `json:"updateAuthData,omitempty"` + RuntimeFlags string `json:"runtimeFlags,omitempty"` } type SinkData struct { @@ -177,6 +179,7 @@ type SinkData struct { TransformFunction string `json:"transformFunction,omitempty"` TransformFunctionClassName string `json:"transformFunctionClassName,omitempty"` TransformFunctionConfig string `json:"transformFunctionConfig,omitempty"` + RuntimeFlags string `json:"runtimeFlags,omitempty"` SinkConf *SinkConfig `json:"-,omitempty"` } From 26e80855bb9848d41781840368e53d74f7df1102 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 18 Apr 2024 06:08:33 -0700 Subject: [PATCH 336/348] [improve] Use physical address information in connection pool key (#1206) ### Motivation Migrate /~https://github.com/apache/pulsar/pull/22085/ and (parts of) /~https://github.com/apache/pulsar-client-cpp/pull/411/ over to the Go client. Context for this idea [here](/~https://github.com/apache/pulsar/pull/22085/files#r1497008116). Golang client support for blue-green migration needs the connection pool to differentiate between connections with the same logical address, but different physical addresses. Otherwise, the wrong connection might be used by the client, in effect pointing to the old cluster, instead of the new one. ### Modifications The connection pool maintains a map of connections, keyed by their logical address and a random connection id. This PR proposes including the physical address in the key also, therefore allowing the upper layer to differentiate between connections with identical logical addresses, but different physical addresses. In addition to this change, the test setup had to be fixed to address breakages in `TestRetryWithMultipleHosts` and `TestReaderWithMultiHosts`. All tests in the repository are using a local standalone setup currently. This unusual configuration has broker lookup operations reply with flag `proxyThroughServiceUrl=true` ([ref](/~https://github.com/apache/pulsar/blob/e7c2a75473b545134a3b292ae0e87a79d65cb756/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java#L369)). This in turn has the Go lookup service attempt a name resolution of the configured service addresses ([ref](/~https://github.com/apache/pulsar-client-go/blob/c3e94e243a730ae22d59bf9d330c4539733b7eef/pulsar/internal/lookup_service.go#L124)). The resolver picks addresses in round-robin mode. Because these two tests use a correct (reachable) broker address _and_ an unreachable address, the resolved address can end up pointing to the unreachable address. The connection pool is then corrupted with a logically invalid entry, causing the tests to fail: | Logical Address | Physical Address | Notes | | --------------- | ---------------- | ----- | | reachable-broker | reachable-broker | Valid | | unreachable-broker | unreachable-broker | Valid, but currently unusable | | reachable-broker | unreachable-broker | *Invalid entry* | To address the issue: - Switch the test setup to a more common cluster configuration. File `integration-tests/clustered/docker-compose.yml` instructs how this setup should look like. - Migrate the tests to separate files and test suites. New test files `pulsar/client_impl_clustered_test.go` and `pulsar/reader_clustered_test.go` contain Go tag `clustered`, allowing them to be ignored during the standalone test runs by virtue of the Go build process. - Add script `run-ci-clustered.sh`, specifying the "clustered" tests to run. - Changes in the `Makefile` add targets `make test_clustered` `make test_standalone` to run the respective test suites independently, while allowing `make test` to run all the tests, as before. - `Dockerfile` and `run-ci.sh` are modified to run the Go build process in the container build, such that it does not need to be run again in the new `run-ci-clustered.sh` script. The image is locally consumed by the tests only and is not published, so there is no risk of contaminating users. --- Dockerfile | 13 ++ Makefile | 10 +- .../clustered/docker-compose.yml | 167 ++++++++++++++++++ pulsar/client_impl_clustered_test.go | 89 ++++++++++ pulsar/client_impl_test.go | 55 ------ pulsar/internal/connection_pool.go | 7 +- pulsar/reader_clustered_test.go | 88 +++++++++ pulsar/reader_test.go | 52 ------ scripts/run-ci-clustered.sh | 23 +++ scripts/run-ci.sh | 10 -- 10 files changed, 393 insertions(+), 121 deletions(-) create mode 100644 integration-tests/clustered/docker-compose.yml create mode 100644 pulsar/client_impl_clustered_test.go create mode 100644 pulsar/reader_clustered_test.go create mode 100755 scripts/run-ci-clustered.sh diff --git a/Dockerfile b/Dockerfile index 818a106a9f..51e35f0a67 100644 --- a/Dockerfile +++ b/Dockerfile @@ -42,3 +42,16 @@ COPY integration-tests/conf/.htpasswd \ COPY . /pulsar/pulsar-client-go ENV PULSAR_EXTRA_OPTS="-Dpulsar.auth.basic.conf=/pulsar/conf/.htpasswd" + +WORKDIR /pulsar/pulsar-client-go + +ENV GOPATH=/pulsar/go +ENV GOCACHE=/tmp/go-cache + +# Install dependencies +RUN go mod download + +# Basic compilation +RUN go build ./pulsar +RUN go build ./pulsaradmin +RUN go build -o bin/pulsar-perf ./perf diff --git a/Makefile b/Makefile index 62e44166cc..df4d539d49 100644 --- a/Makefile +++ b/Makefile @@ -44,9 +44,17 @@ container: --build-arg PULSAR_IMAGE="${PULSAR_IMAGE}" \ --build-arg ARCH="${CONTAINER_ARCH}" . -test: container +test: container test_standalone test_clustered + +test_standalone: container docker run -i ${IMAGE_NAME} bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci.sh" +test_clustered: container + PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/clustered/docker-compose.yml up -d || true + until curl http://localhost:8080/metrics > /dev/null 2>&1; do sleep 1; done + docker run --network "clustered_pulsar" -i ${IMAGE_NAME} bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci-clustered.sh" + PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/clustered/docker-compose.yml down + clean: docker rmi --force $(IMAGE_NAME) || true rm bin/* diff --git a/integration-tests/clustered/docker-compose.yml b/integration-tests/clustered/docker-compose.yml new file mode 100644 index 0000000000..cce8edddad --- /dev/null +++ b/integration-tests/clustered/docker-compose.yml @@ -0,0 +1,167 @@ +# 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. + +version: '3' +networks: + pulsar: + driver: bridge +services: + # Start ZooKeeper + zookeeper: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: zookeeper + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - PULSAR_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + command: > + bash -c "bin/apply-config-from-env.py conf/zookeeper.conf && \ + bin/generate-zookeeper-config.sh conf/zookeeper.conf && \ + exec bin/pulsar zookeeper" + healthcheck: + test: ["CMD", "bin/pulsar-zookeeper-ruok.sh"] + interval: 10s + timeout: 5s + retries: 30 + + # Initialize cluster metadata + pulsar-init: + container_name: pulsar-init + hostname: pulsar-init + image: apachepulsar/pulsar:${PULSAR_VERSION} + networks: + - pulsar + environment: + - PULSAR_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + command: > + bin/pulsar initialize-cluster-metadata \ + --cluster cluster-a \ + --zookeeper zookeeper:2181 \ + --configuration-store zookeeper:2181 \ + --web-service-url http://broker-1:8080 \ + --broker-service-url pulsar://broker-1:6650 + depends_on: + zookeeper: + condition: service_healthy + + # Start bookie + bookie: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: bookie + restart: on-failure + networks: + - pulsar + environment: + - clusterName=cluster-a + - zkServers=zookeeper:2181 + - metadataServiceUri=metadata-store:zk:zookeeper:2181 + - advertisedAddress=bookie + - BOOKIE_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + depends_on: + zookeeper: + condition: service_healthy + pulsar-init: + condition: service_completed_successfully + command: bash -c "bin/apply-config-from-env.py conf/bookkeeper.conf && exec bin/pulsar bookie" + + proxy: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: proxy + hostname: proxy + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - zookeeperServers=zookeeper:2181 + - clusterName=cluster-a + - PULSAR_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + ports: + - "8080:8080" + - "6650:6650" + depends_on: + broker-1: + condition: service_healthy + broker-2: + condition: service_healthy + command: bash -c "bin/apply-config-from-env.py conf/proxy.conf && exec bin/pulsar proxy" + + # Start broker 1 + broker-1: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: broker-1 + hostname: broker-1 + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - zookeeperServers=zookeeper:2181 + - clusterName=cluster-a + - managedLedgerDefaultEnsembleSize=1 + - managedLedgerDefaultWriteQuorum=1 + - managedLedgerDefaultAckQuorum=1 + - advertisedAddress=broker-1 + - internalListenerName=internal + - advertisedListeners=internal:pulsar://broker-1:6650 + - PULSAR_MEM=-Xms256m -Xmx256m -XX:MaxDirectMemorySize=56m + - PULSAR_PREFIX_defaultNumberOfNamespaceBundles=1 + depends_on: + zookeeper: + condition: service_healthy + bookie: + condition: service_started + command: bash -c "bin/apply-config-from-env.py conf/broker.conf && exec bin/pulsar broker" + healthcheck: + test: ["CMD-SHELL", "curl -f http://localhost:8080/metrics || exit 1"] + interval: 10s + timeout: 5s + retries: 30 + + # Start broker 2 + broker-2: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: broker-2 + hostname: broker-2 + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - zookeeperServers=zookeeper:2181 + - clusterName=cluster-a + - managedLedgerDefaultEnsembleSize=1 + - managedLedgerDefaultWriteQuorum=1 + - managedLedgerDefaultAckQuorum=1 + - advertisedAddress=broker-2 + - internalListenerName=internal + - advertisedListeners=internal:pulsar://broker-2:6650 + - PULSAR_MEM=-Xms256m -Xmx256m -XX:MaxDirectMemorySize=56m + - PULSAR_PREFIX_defaultNumberOfNamespaceBundles=1 + depends_on: + zookeeper: + condition: service_healthy + bookie: + condition: service_started + command: bash -c "bin/apply-config-from-env.py conf/broker.conf && exec bin/pulsar broker" + healthcheck: + test: ["CMD-SHELL", "curl -f http://localhost:8080/metrics || exit 1"] + interval: 10s + timeout: 5s + retries: 30 diff --git a/pulsar/client_impl_clustered_test.go b/pulsar/client_impl_clustered_test.go new file mode 100644 index 0000000000..e572c7741c --- /dev/null +++ b/pulsar/client_impl_clustered_test.go @@ -0,0 +1,89 @@ +//go:build clustered + +// 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 pulsar + +import ( + "context" + "fmt" + "testing" + + "github.com/stretchr/testify/suite" +) + +type clientClusteredTestSuite struct { + suite.Suite +} + +func TestClientClusteredTestSuite(t *testing.T) { + suite.Run(t, new(clientClusteredTestSuite)) +} + +func (suite *clientClusteredTestSuite) TestRetryWithMultipleHosts() { + req := suite.Require() + // Multi hosts included an unreached port and the actual port for verify retry logic + client, err := NewClient(ClientOptions{ + URL: "pulsar://broker-1:6600,broker-1:6650", + }) + req.NoError(err) + defer client.Close() + + topic := "persistent://public/default/retry-multiple-hosts-" + generateRandomName() + + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + }) + req.NoError(err) + defer producer.Close() + + ctx := context.Background() + var msgIDs [][]byte + + for i := 0; i < 10; i++ { + if msgID, err := producer.Send(ctx, &ProducerMessage{ + Payload: []byte(fmt.Sprintf("hello-%d", i)), + }); err != nil { + req.NoError(err) + } else { + req.NotNil(msgID) + msgIDs = append(msgIDs, msgID.Serialize()) + } + } + + req.Equal(10, len(msgIDs)) + + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "retry-multi-hosts-sub", + Type: Shared, + SubscriptionInitialPosition: SubscriptionPositionEarliest, + }) + req.NoError(err) + defer consumer.Close() + + for i := 0; i < 10; i++ { + msg, err := consumer.Receive(context.Background()) + req.NoError(err) + req.Contains(msgIDs, msg.ID().Serialize()) + consumer.Ack(msg) + } + + err = consumer.Unsubscribe() + req.NoError(err) +} diff --git a/pulsar/client_impl_test.go b/pulsar/client_impl_test.go index 78dc1cae68..5b6b8f1ff3 100644 --- a/pulsar/client_impl_test.go +++ b/pulsar/client_impl_test.go @@ -570,61 +570,6 @@ func anonymousNamespacePolicy() map[string]interface{} { } } -func TestRetryWithMultipleHosts(t *testing.T) { - // Multi hosts included an unreached port and the actual port for verify retry logic - client, err := NewClient(ClientOptions{ - URL: "pulsar://localhost:6600,localhost:6650", - }) - - assert.Nil(t, err) - defer client.Close() - - topic := "persistent://public/default/retry-multiple-hosts-" + generateRandomName() - - producer, err := client.CreateProducer(ProducerOptions{ - Topic: topic, - }) - - assert.Nil(t, err) - defer producer.Close() - - ctx := context.Background() - var msgIDs [][]byte - - for i := 0; i < 10; i++ { - if msgID, err := producer.Send(ctx, &ProducerMessage{ - Payload: []byte(fmt.Sprintf("hello-%d", i)), - }); err != nil { - assert.Nil(t, err) - } else { - assert.NotNil(t, msgID) - msgIDs = append(msgIDs, msgID.Serialize()) - } - } - - assert.Equal(t, 10, len(msgIDs)) - - consumer, err := client.Subscribe(ConsumerOptions{ - Topic: topic, - SubscriptionName: "retry-multi-hosts-sub", - Type: Shared, - SubscriptionInitialPosition: SubscriptionPositionEarliest, - }) - assert.Nil(t, err) - defer consumer.Close() - - for i := 0; i < 10; i++ { - msg, err := consumer.Receive(context.Background()) - assert.Nil(t, err) - assert.Contains(t, msgIDs, msg.ID().Serialize()) - consumer.Ack(msg) - } - - err = consumer.Unsubscribe() - assert.Nil(t, err) - -} - func TestHTTPSConnectionCAError(t *testing.T) { client, err := NewClient(ClientOptions{ URL: webServiceURLTLS, diff --git a/pulsar/internal/connection_pool.go b/pulsar/internal/connection_pool.go index 6ff7991992..3d718b75d9 100644 --- a/pulsar/internal/connection_pool.go +++ b/pulsar/internal/connection_pool.go @@ -79,7 +79,8 @@ func NewConnectionPool( } func (p *connectionPool) GetConnection(logicalAddr *url.URL, physicalAddr *url.URL) (Connection, error) { - key := p.getMapKey(logicalAddr) + p.log.WithField("logicalAddr", logicalAddr).WithField("physicalAddr", physicalAddr).Debug("Getting pooled connection") + key := p.getMapKey(logicalAddr, physicalAddr) p.Lock() conn, ok := p.connections[key] @@ -133,13 +134,13 @@ func (p *connectionPool) Close() { p.Unlock() } -func (p *connectionPool) getMapKey(addr *url.URL) string { +func (p *connectionPool) getMapKey(logicalAddr *url.URL, physicalAddr *url.URL) string { cnt := atomic.AddInt32(&p.roundRobinCnt, 1) if cnt < 0 { cnt = -cnt } idx := cnt % p.maxConnectionsPerHost - return fmt.Sprint(addr.Host, '-', idx) + return fmt.Sprint(logicalAddr.Host, "-", physicalAddr.Host, "-", idx) } func (p *connectionPool) checkAndCleanIdleConnections(maxIdleTime time.Duration) { diff --git a/pulsar/reader_clustered_test.go b/pulsar/reader_clustered_test.go new file mode 100644 index 0000000000..db2dc6f130 --- /dev/null +++ b/pulsar/reader_clustered_test.go @@ -0,0 +1,88 @@ +//go:build clustered + +// 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 pulsar + +import ( + "context" + "fmt" + "testing" + + "github.com/stretchr/testify/suite" +) + +type ReaderClusteredTestSuite struct { + suite.Suite +} + +func TestReaderClusteredTestSuite(t *testing.T) { + suite.Run(t, new(ReaderClusteredTestSuite)) +} + +func (suite *ReaderClusteredTestSuite) TestReaderWithMultipleHosts() { + req := suite.Require() + + // Multi hosts included an unreached port and the actual port for verify retry logic + client, err := NewClient(ClientOptions{ + URL: "pulsar://broker-1:6600,broker-1:6650", + }) + req.NoError(err) + defer client.Close() + + topic := newTopicName() + ctx := context.Background() + + // create producer + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + DisableBatching: true, + }) + req.NoError(err) + defer producer.Close() + + // send 10 messages + for i := 0; i < 10; i++ { + msgID, err := producer.Send(ctx, &ProducerMessage{ + Payload: []byte(fmt.Sprintf("hello-%d", i)), + }) + req.NoError(err) + req.NotNil(msgID) + } + + // create reader on 5th message (not included) + reader, err := client.CreateReader(ReaderOptions{ + Topic: topic, + StartMessageID: EarliestMessageID(), + }) + req.NoError(err) + defer reader.Close() + + i := 0 + for reader.HasNext() { + msg, err := reader.Next(context.Background()) + req.NoError(err) + + expectMsg := fmt.Sprintf("hello-%d", i) + req.Equal([]byte(expectMsg), msg.Payload()) + + i++ + } + + req.Equal(10, i) +} diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index 78c222dac7..93787d106c 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -724,58 +724,6 @@ func TestReaderLatestInclusiveHasNext(t *testing.T) { assert.False(t, reader.HasNext()) } -func TestReaderWithMultiHosts(t *testing.T) { - // Multi hosts included an unreached port and the actual port for verify retry logic - client, err := NewClient(ClientOptions{ - URL: "pulsar://localhost:6600,localhost:6650", - }) - - assert.Nil(t, err) - defer client.Close() - - topic := newTopicName() - ctx := context.Background() - - // create producer - producer, err := client.CreateProducer(ProducerOptions{ - Topic: topic, - DisableBatching: true, - }) - assert.Nil(t, err) - defer producer.Close() - - // send 10 messages - for i := 0; i < 10; i++ { - msgID, err := producer.Send(ctx, &ProducerMessage{ - Payload: []byte(fmt.Sprintf("hello-%d", i)), - }) - assert.NoError(t, err) - assert.NotNil(t, msgID) - } - - // create reader on 5th message (not included) - reader, err := client.CreateReader(ReaderOptions{ - Topic: topic, - StartMessageID: EarliestMessageID(), - }) - - assert.Nil(t, err) - defer reader.Close() - - i := 0 - for reader.HasNext() { - msg, err := reader.Next(context.Background()) - assert.NoError(t, err) - - expectMsg := fmt.Sprintf("hello-%d", i) - assert.Equal(t, []byte(expectMsg), msg.Payload()) - - i++ - } - - assert.Equal(t, 10, i) -} - func TestProducerReaderRSAEncryption(t *testing.T) { client, err := NewClient(ClientOptions{ URL: lookupURL, diff --git a/scripts/run-ci-clustered.sh b/scripts/run-ci-clustered.sh new file mode 100755 index 0000000000..5e4f36fa4c --- /dev/null +++ b/scripts/run-ci-clustered.sh @@ -0,0 +1,23 @@ +#!/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. + +set -e -x + +go test -race -coverprofile=/tmp/coverage -timeout=5m -tags clustered -v -run 'Test.*ClusteredTestSuite' -v ./pulsar +go tool cover -html=/tmp/coverage -o coverage.html + diff --git a/scripts/run-ci.sh b/scripts/run-ci.sh index 83246a39f0..e7a6d79ece 100755 --- a/scripts/run-ci.sh +++ b/scripts/run-ci.sh @@ -19,16 +19,6 @@ set -e -x -export GOPATH=/pulsar/go -export GOCACHE=/tmp/go-cache - -# Install dependencies -go mod download - -# Basic compilation -go build ./pulsar -go build -o bin/pulsar-perf ./perf - scripts/pulsar-test-service-start.sh go test -race -coverprofile=/tmp/coverage -timeout=20m -v ./... From 13ecca24ffb6b65ac7ec36da53ecb67479518647 Mon Sep 17 00:00:00 2001 From: zhou zhuohan <843520313@qq.com> Date: Mon, 22 Apr 2024 21:23:49 +0800 Subject: [PATCH 337/348] [improve] add a lint-docker command in makefile (#1207) Co-authored-by: ninjazhou --- Makefile | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/Makefile b/Makefile index df4d539d49..cdae8a5983 100644 --- a/Makefile +++ b/Makefile @@ -38,6 +38,12 @@ lint: bin/golangci-lint bin/golangci-lint: GOBIN=$(shell pwd)/bin go install github.com/golangci/golangci-lint/cmd/golangci-lint@v1.51.2 +# an alternative to above `make lint` command +# use golangCi-lint docker to avoid local golang env issues +# https://golangci-lint.run/welcome/install/ +lint-docker: + docker run --rm -v $(shell pwd):/app -w /app golangci/golangci-lint:v1.51.2 golangci-lint run -v + container: docker build -t ${IMAGE_NAME} \ --build-arg GO_VERSION="${GO_VERSION}" \ From b4d45cd360599b52e3f160db9d39d619889c556a Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Mon, 22 Apr 2024 21:25:00 +0800 Subject: [PATCH 338/348] [Improve] Add admin api GetLeaderBroker (#1203) ### Motivation To keep consistent with the [Java client](/~https://github.com/apache/pulsar/pull/9799). ### Modifications Add `GetLeaderBroker` interface. --- pulsaradmin/pkg/admin/brokers.go | 12 ++++++++++++ pulsaradmin/pkg/admin/brokers_test.go | 16 ++++++++++++++++ pulsaradmin/pkg/utils/data.go | 5 +++++ 3 files changed, 33 insertions(+) diff --git a/pulsaradmin/pkg/admin/brokers.go b/pulsaradmin/pkg/admin/brokers.go index e178610c30..650fab8e0b 100644 --- a/pulsaradmin/pkg/admin/brokers.go +++ b/pulsaradmin/pkg/admin/brokers.go @@ -58,6 +58,9 @@ type Brokers interface { // HealthCheckWithTopicVersion run a health check on the broker HealthCheckWithTopicVersion(utils.TopicVersion) error + + // GetLeaderBroker get the information of the leader broker. + GetLeaderBroker() (utils.BrokerInfo, error) } type broker struct { @@ -162,3 +165,12 @@ func (b *broker) HealthCheckWithTopicVersion(topicVersion utils.TopicVersion) er } return nil } +func (b *broker) GetLeaderBroker() (utils.BrokerInfo, error) { + endpoint := b.pulsar.endpoint(b.basePath, "/leaderBroker") + var brokerInfo utils.BrokerInfo + err := b.pulsar.Client.Get(endpoint, &brokerInfo) + if err != nil { + return brokerInfo, err + } + return brokerInfo, nil +} diff --git a/pulsaradmin/pkg/admin/brokers_test.go b/pulsaradmin/pkg/admin/brokers_test.go index d48ce7cbd4..9767975905 100644 --- a/pulsaradmin/pkg/admin/brokers_test.go +++ b/pulsaradmin/pkg/admin/brokers_test.go @@ -42,3 +42,19 @@ func TestBrokerHealthCheckWithTopicVersion(t *testing.T) { err = admin.Brokers().HealthCheckWithTopicVersion(utils.TopicVersionV2) assert.NoError(t, err) } + +func TestGetLeaderBroker(t *testing.T) { + readFile, err := os.ReadFile("../../../integration-tests/tokens/admin-token") + assert.NoError(t, err) + cfg := &config.Config{ + Token: string(readFile), + } + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + leaderBroker, err := admin.Brokers().GetLeaderBroker() + assert.NoError(t, err) + assert.NotNil(t, leaderBroker) + assert.NotEmpty(t, leaderBroker.ServiceURL) + assert.NotEmpty(t, leaderBroker.BrokerID) +} diff --git a/pulsaradmin/pkg/utils/data.go b/pulsaradmin/pkg/utils/data.go index 1e67e3c71c..616079127f 100644 --- a/pulsaradmin/pkg/utils/data.go +++ b/pulsaradmin/pkg/utils/data.go @@ -477,6 +477,11 @@ type GetStatsOptions struct { ExcludeConsumers bool `json:"exclude_consumers"` } +type BrokerInfo struct { + BrokerID string `json:"brokerId"` + ServiceURL string `json:"serviceUrl"` +} + type TopicVersion string const ( From 458defe3ca66ba87664966bcafd58d1350b46acf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Apr 2024 09:06:38 -0700 Subject: [PATCH 339/348] chore(deps): bump golang.org/x/net from 0.17.0 to 0.23.0 (#1209) Bumps [golang.org/x/net](/~https://github.com/golang/net) from 0.17.0 to 0.23.0. - [Commits](/~https://github.com/golang/net/compare/v0.17.0...v0.23.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 8 ++++---- go.sum | 16 ++++++++-------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/go.mod b/go.mod index 511648711b..5225551207 100644 --- a/go.mod +++ b/go.mod @@ -58,10 +58,10 @@ require ( github.com/prometheus/procfs v0.6.0 // indirect github.com/spf13/pflag v1.0.5 // indirect github.com/stretchr/objx v0.4.0 // indirect - golang.org/x/net v0.17.0 // indirect - golang.org/x/sys v0.13.0 // indirect - golang.org/x/term v0.13.0 // indirect - golang.org/x/text v0.13.0 // indirect + golang.org/x/net v0.23.0 // indirect + golang.org/x/sys v0.18.0 // indirect + golang.org/x/term v0.18.0 // indirect + golang.org/x/text v0.14.0 // indirect google.golang.org/appengine v1.6.7 // indirect gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect diff --git a/go.sum b/go.sum index 0a768a0eff..f48b384ace 100644 --- a/go.sum +++ b/go.sum @@ -355,8 +355,8 @@ golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= -golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= +golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= +golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -416,19 +416,19 @@ golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210819135213-f52c844e1c1c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= -golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= +golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.13.0 h1:bb+I9cTfFazGW51MZqBVmZy7+JEJMouUHTUSKVQLBek= -golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= +golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= +golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.13.0 h1:ablQoSUd0tRdKxZewP80B+BaqeKJuVhuRxj/dkrun3k= -golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= +golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0 h1:/5xXl8Y5W96D+TtHSlonuFqGHIWVuyCkGJLwGh9JJFs= From 86054c5f0f4aa211a0785ca2b040854629a0b196 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 23 Apr 2024 09:30:50 -0700 Subject: [PATCH 340/348] [improve] PIP-307: Use assigned broker URL hints during broker reconnection (#1208) --- Makefile | 10 +- .../docker-compose.yml | 181 ++ pulsar/client_impl.go | 13 +- pulsar/consumer_partition.go | 58 +- pulsar/consumer_test.go | 8 +- pulsar/extensible_load_manager_test.go | 231 +++ pulsar/internal/connection.go | 19 +- pulsar/internal/lookup_service.go | 94 +- pulsar/internal/pulsar_proto/PulsarApi.pb.go | 1728 +++++++++-------- pulsar/internal/pulsar_proto/PulsarApi.proto | 4 + pulsar/producer_partition.go | 68 +- pulsar/producer_test.go | 8 +- pulsar/reader_test.go | 8 +- scripts/run-ci-extensible-load-manager.sh | 23 + 14 files changed, 1504 insertions(+), 949 deletions(-) create mode 100644 integration-tests/extensible-load-manager/docker-compose.yml create mode 100644 pulsar/extensible_load_manager_test.go create mode 100755 scripts/run-ci-extensible-load-manager.sh diff --git a/Makefile b/Makefile index cdae8a5983..4e119fc366 100644 --- a/Makefile +++ b/Makefile @@ -50,17 +50,23 @@ container: --build-arg PULSAR_IMAGE="${PULSAR_IMAGE}" \ --build-arg ARCH="${CONTAINER_ARCH}" . -test: container test_standalone test_clustered +test: container test_standalone test_clustered test_extensible_load_manager test_standalone: container docker run -i ${IMAGE_NAME} bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci.sh" test_clustered: container - PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/clustered/docker-compose.yml up -d || true + PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/clustered/docker-compose.yml up -d until curl http://localhost:8080/metrics > /dev/null 2>&1; do sleep 1; done docker run --network "clustered_pulsar" -i ${IMAGE_NAME} bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci-clustered.sh" PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/clustered/docker-compose.yml down +test_extensible_load_manager: container + PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/extensible-load-manager/docker-compose.yml up -d + until curl http://localhost:8080/metrics > /dev/null 2>&1; do sleep 1; done + docker run --network "extensible-load-manager_pulsar" -i ${IMAGE_NAME} bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci-extensible-load-manager.sh" + PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/extensible-load-manager/docker-compose.yml down + clean: docker rmi --force $(IMAGE_NAME) || true rm bin/* diff --git a/integration-tests/extensible-load-manager/docker-compose.yml b/integration-tests/extensible-load-manager/docker-compose.yml new file mode 100644 index 0000000000..935afdcefa --- /dev/null +++ b/integration-tests/extensible-load-manager/docker-compose.yml @@ -0,0 +1,181 @@ +# 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. + +version: '3' +networks: + pulsar: + driver: bridge +services: + # Start ZooKeeper + zookeeper: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: zookeeper + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - PULSAR_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + command: > + bash -c "bin/apply-config-from-env.py conf/zookeeper.conf && \ + bin/generate-zookeeper-config.sh conf/zookeeper.conf && \ + exec bin/pulsar zookeeper" + healthcheck: + test: ["CMD", "bin/pulsar-zookeeper-ruok.sh"] + interval: 10s + timeout: 5s + retries: 30 + + # Initialize cluster metadata + pulsar-init: + container_name: pulsar-init + hostname: pulsar-init + image: apachepulsar/pulsar:${PULSAR_VERSION} + networks: + - pulsar + environment: + - PULSAR_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + command: > + bin/pulsar initialize-cluster-metadata \ + --cluster cluster-a \ + --zookeeper zookeeper:2181 \ + --configuration-store zookeeper:2181 \ + --web-service-url http://broker-1:8080 \ + --broker-service-url pulsar://broker-1:6650 + depends_on: + zookeeper: + condition: service_healthy + + # Start bookie + bookie: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: bookie + restart: on-failure + networks: + - pulsar + environment: + - clusterName=cluster-a + - zkServers=zookeeper:2181 + - metadataServiceUri=metadata-store:zk:zookeeper:2181 + - advertisedAddress=bookie + - BOOKIE_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + depends_on: + zookeeper: + condition: service_healthy + pulsar-init: + condition: service_completed_successfully + command: bash -c "bin/apply-config-from-env.py conf/bookkeeper.conf && exec bin/pulsar bookie" + + proxy: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: proxy + hostname: proxy + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - zookeeperServers=zookeeper:2181 + - clusterName=cluster-a + - PULSAR_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + ports: + - "8080:8080" + - "6650:6650" + depends_on: + broker-1: + condition: service_healthy + broker-2: + condition: service_healthy + command: bash -c "bin/apply-config-from-env.py conf/proxy.conf && exec bin/pulsar proxy" + + # Start broker 1 + broker-1: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: broker-1 + hostname: broker-1 + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - zookeeperServers=zookeeper:2181 + - clusterName=cluster-a + - managedLedgerDefaultEnsembleSize=1 + - managedLedgerDefaultWriteQuorum=1 + - managedLedgerDefaultAckQuorum=1 + - advertisedAddress=broker-1 + - internalListenerName=internal + - advertisedListeners=internal:pulsar://broker-1:6650 + - PULSAR_MEM=-Xms256m -Xmx256m -XX:MaxDirectMemorySize=56m + # Load Manager. Here uses the extensible load balancer, sets the unloading strategy to TransferShedder, and enables debug mode. + - loadManagerClassName=org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl + - loadBalancerLoadSheddingStrategy=org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder + - loadBalancerSheddingEnabled=false + - loadBalancerDebugModeEnabled=true + - clusterMigrationCheckDurationSeconds=1 + - brokerServiceCompactionThresholdInBytes=1000000 + - PULSAR_PREFIX_defaultNumberOfNamespaceBundles=1 + depends_on: + zookeeper: + condition: service_healthy + bookie: + condition: service_started + command: bash -c "bin/apply-config-from-env.py conf/broker.conf && exec bin/pulsar broker" + healthcheck: + test: ["CMD-SHELL", "curl -f http://localhost:8080/metrics || exit 1"] + interval: 10s + timeout: 5s + retries: 30 + + # Start broker 2 + broker-2: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: broker-2 + hostname: broker-2 + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - zookeeperServers=zookeeper:2181 + - clusterName=cluster-a + - managedLedgerDefaultEnsembleSize=1 + - managedLedgerDefaultWriteQuorum=1 + - managedLedgerDefaultAckQuorum=1 + - advertisedAddress=broker-2 + - internalListenerName=internal + - advertisedListeners=internal:pulsar://broker-2:6650 + - PULSAR_MEM=-Xms256m -Xmx256m -XX:MaxDirectMemorySize=56m + # Load Manager. Here uses the extensible load balancer, sets the unloading strategy to TransferShedder, and enables debug mode. + - loadManagerClassName=org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl + - loadBalancerLoadSheddingStrategy=org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder + - loadBalancerSheddingEnabled=false + - loadBalancerDebugModeEnabled=true + - clusterMigrationCheckDurationSeconds=1 + - brokerServiceCompactionThresholdInBytes=1000000 + - PULSAR_PREFIX_defaultNumberOfNamespaceBundles=1 + depends_on: + zookeeper: + condition: service_healthy + bookie: + condition: service_started + command: bash -c "bin/apply-config-from-env.py conf/broker.conf && exec bin/pulsar broker" + healthcheck: + test: ["CMD-SHELL", "curl -f http://localhost:8080/metrics || exit 1"] + interval: 10s + timeout: 5s + retries: 30 diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index 65aed3b963..8471e1e39a 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -49,6 +49,7 @@ type client struct { memLimit internal.MemoryLimitController closeOnce sync.Once operationTimeout time.Duration + tlsEnabled bool log log.Logger } @@ -166,6 +167,7 @@ func newClient(options ClientOptions) (Client, error) { metrics: metrics, memLimit: internal.NewMemoryLimitController(memLimitBytes, defaultMemoryLimitTriggerThreshold), operationTimeout: operationTimeout, + tlsEnabled: tlsConfig != nil, } serviceNameResolver := internal.NewPulsarServiceNameResolver(url) @@ -174,7 +176,7 @@ func newClient(options ClientOptions) (Client, error) { switch url.Scheme { case "pulsar", "pulsar+ssl": c.lookupService = internal.NewLookupService(c.rpcClient, url, serviceNameResolver, - tlsConfig != nil, options.ListenerName, logger, metrics) + c.tlsEnabled, options.ListenerName, logger, metrics) case "http", "https": httpClient, err := internal.NewHTTPClient(url, serviceNameResolver, tlsConfig, operationTimeout, logger, metrics, authProvider) @@ -183,7 +185,7 @@ func newClient(options ClientOptions) (Client, error) { err.Error())) } c.lookupService = internal.NewHTTPLookupService(httpClient, url, serviceNameResolver, - tlsConfig != nil, logger, metrics) + c.tlsEnabled, logger, metrics) default: return nil, newError(InvalidConfiguration, fmt.Sprintf("Invalid URL scheme '%s'", url.Scheme)) } @@ -275,3 +277,10 @@ func (c *client) Close() { c.lookupService.Close() }) } + +func (c *client) selectServiceURL(brokerServiceURL, brokerServiceURLTLS string) string { + if c.tlsEnabled { + return brokerServiceURLTLS + } + return brokerServiceURL +} diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 162565b2a9..dc01e69285 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -158,7 +158,7 @@ type partitionConsumer struct { eventsCh chan interface{} connectedCh chan struct{} - connectClosedCh chan connectionClosed + connectClosedCh chan *connectionClosed closeCh chan struct{} clearQueueCh chan func(id *trackingMessageID) @@ -326,7 +326,7 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon startMessageID: atomicMessageID{msgID: options.startMessageID}, connectedCh: make(chan struct{}), messageCh: messageCh, - connectClosedCh: make(chan connectionClosed, 10), + connectClosedCh: make(chan *connectionClosed, 10), closeCh: make(chan struct{}), clearQueueCh: make(chan func(id *trackingMessageID)), compressionProviders: sync.Map{}, @@ -370,7 +370,7 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon pc.nackTracker = newNegativeAcksTracker(pc, options.nackRedeliveryDelay, options.nackBackoffPolicy, pc.log) - err := pc.grabConn() + err := pc.grabConn("") if err != nil { pc.log.WithError(err).Error("Failed to create consumer") pc.nackTracker.Close() @@ -1358,10 +1358,17 @@ func createEncryptionContext(msgMeta *pb.MessageMetadata) *EncryptionContext { return &encCtx } -func (pc *partitionConsumer) ConnectionClosed() { +func (pc *partitionConsumer) ConnectionClosed(closeConsumer *pb.CommandCloseConsumer) { // Trigger reconnection in the consumer goroutine pc.log.Debug("connection closed and send to connectClosedCh") - pc.connectClosedCh <- connectionClosed{} + var assignedBrokerURL string + if closeConsumer != nil { + assignedBrokerURL = pc.client.selectServiceURL( + closeConsumer.GetAssignedBrokerServiceUrl(), closeConsumer.GetAssignedBrokerServiceUrlTls()) + } + pc.connectClosedCh <- &connectionClosed{ + assignedBrokerURL: assignedBrokerURL, + } } // Flow command gives additional permits to send messages to the consumer. @@ -1566,9 +1573,9 @@ func (pc *partitionConsumer) runEventsLoop() { case <-pc.closeCh: pc.log.Info("close consumer, exit reconnect") return - case <-pc.connectClosedCh: + case connectionClosed := <-pc.connectClosedCh: pc.log.Debug("runEventsLoop will reconnect") - pc.reconnectToBroker() + pc.reconnectToBroker(connectionClosed) } } }() @@ -1652,7 +1659,7 @@ func (pc *partitionConsumer) internalClose(req *closeRequest) { close(pc.closeCh) } -func (pc *partitionConsumer) reconnectToBroker() { +func (pc *partitionConsumer) reconnectToBroker(connectionClosed *connectionClosed) { var maxRetry int if pc.options.maxReconnectToBroker == nil { @@ -1673,13 +1680,22 @@ func (pc *partitionConsumer) reconnectToBroker() { return } - if pc.options.backoffPolicy == nil { + var assignedBrokerURL string + + if connectionClosed != nil && connectionClosed.HasURL() { + delayReconnectTime = 0 + assignedBrokerURL = connectionClosed.assignedBrokerURL + connectionClosed = nil // Attempt connecting to the assigned broker just once + } else if pc.options.backoffPolicy == nil { delayReconnectTime = defaultBackoff.Next() } else { delayReconnectTime = pc.options.backoffPolicy.Next() } - pc.log.Info("Reconnecting to broker in ", delayReconnectTime) + pc.log.WithFields(log.Fields{ + "assignedBrokerURL": assignedBrokerURL, + "delayReconnectTime": delayReconnectTime, + }).Info("Reconnecting to broker") time.Sleep(delayReconnectTime) // double check @@ -1689,7 +1705,7 @@ func (pc *partitionConsumer) reconnectToBroker() { return } - err := pc.grabConn() + err := pc.grabConn(assignedBrokerURL) if err == nil { // Successfully reconnected pc.log.Info("Reconnected consumer to broker") @@ -1713,13 +1729,25 @@ func (pc *partitionConsumer) reconnectToBroker() { } } -func (pc *partitionConsumer) grabConn() error { - lr, err := pc.client.lookupService.Lookup(pc.topic) +func (pc *partitionConsumer) lookupTopic(brokerServiceURL string) (*internal.LookupResult, error) { + if len(brokerServiceURL) == 0 { + lr, err := pc.client.lookupService.Lookup(pc.topic) + if err != nil { + pc.log.WithError(err).Warn("Failed to lookup topic") + return nil, err + } + + pc.log.Debug("Lookup result: ", lr) + return lr, err + } + return pc.client.lookupService.GetBrokerAddress(brokerServiceURL, pc._getConn().IsProxied()) +} + +func (pc *partitionConsumer) grabConn(assignedBrokerURL string) error { + lr, err := pc.lookupTopic(assignedBrokerURL) if err != nil { - pc.log.WithError(err).Warn("Failed to lookup topic") return err } - pc.log.Debugf("Lookup result: %+v", lr) subType := toProtoSubType(pc.options.subscriptionType) initialPosition := toProtoInitialPosition(pc.options.subscriptionInitPos) diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 4a3b532d05..4120ba4bac 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -3796,22 +3796,22 @@ func TestConsumerWithBackoffPolicy(t *testing.T) { partitionConsumerImp := _consumer.(*consumer).consumers[0] // 1 s startTime := time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 2 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) } diff --git a/pulsar/extensible_load_manager_test.go b/pulsar/extensible_load_manager_test.go new file mode 100644 index 0000000000..b3025c390a --- /dev/null +++ b/pulsar/extensible_load_manager_test.go @@ -0,0 +1,231 @@ +//go:build extensible_load_manager + +// 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 pulsar + +import ( + "context" + "fmt" + "net/http" + "sync" + "testing" + "time" + + "github.com/apache/pulsar-client-go/pulsaradmin" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" + + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/suite" + + uAtomic "go.uber.org/atomic" +) + +type ExtensibleLoadManagerTestSuite struct { + suite.Suite +} + +func TestExtensibleLoadManagerTestSuite(t *testing.T) { + suite.Run(t, new(ExtensibleLoadManagerTestSuite)) +} + +const ( + tenant = utils.PUBLICTENANT + namespace = utils.DEFAULTNAMESPACE + + broker1URL = "pulsar://broker-1:6650" + broker2URL = "pulsar://broker-2:6650" + + broker1LookupURL = "broker-1:8080" + broker2LookupURL = "broker-2:8080" +) + +type mockCounter struct { + prometheus.Counter + + count uAtomic.Int32 +} + +func (m *mockCounter) Inc() { + m.count.Inc() +} + +func (suite *ExtensibleLoadManagerTestSuite) TestTopicUnload() { + type topicUnloadTestCase struct { + testCaseName string + adminURL string + clientEndpointFunc func(utils.LookupData) string + unloadEndpointFunc func(utils.LookupData) string + } + for _, scenario := range []topicUnloadTestCase{ + { + testCaseName: "directConnection", + adminURL: "http://broker-1:8080", + clientEndpointFunc: func(lookupResult utils.LookupData) string { + return lookupResult.BrokerURL + }, + unloadEndpointFunc: func(lookupResult utils.LookupData) string { + return lookupResult.HTTPURL + }, + }, + { + testCaseName: "proxyConnection", + adminURL: "http://proxy:8080", + clientEndpointFunc: func(utils.LookupData) string { + return "pulsar://proxy:6650" + }, + unloadEndpointFunc: func(utils.LookupData) string { + return "http://proxy:8080" + }, + }, + } { + suite.T().Run(scenario.testCaseName, func(t *testing.T) { + testTopicUnload(t, scenario.adminURL, scenario.clientEndpointFunc, scenario.unloadEndpointFunc) + }) + } +} + +func testTopicUnload(t *testing.T, adminURL string, + clientEndpointFunc func(utils.LookupData) string, + unloadEndpointFunc func(utils.LookupData) string) { + req := assert.New(t) + + admin, err := pulsaradmin.NewClient(&pulsaradmin.Config{WebServiceURL: adminURL}) + req.NoError(err) + + topicName, err := utils.GetTopicName(newTopicName()) + req.NoError(err) + req.NotNil(topicName) + + err = admin.Topics().Create(*topicName, 0) + req.NoError(err) + + lookupResult, err := admin.Topics().Lookup(*topicName) + req.NoError(err) + req.NotEmpty(lookupResult.BrokerURL) + srcTopicBrokerURL := lookupResult.BrokerURL + req.Contains([...]string{broker1URL, broker2URL}, srcTopicBrokerURL) + + var dstTopicBrokerURL string + if srcTopicBrokerURL == broker1URL { + dstTopicBrokerURL = broker2LookupURL + } else { + dstTopicBrokerURL = broker1LookupURL + } + + bundleRange, err := admin.Topics().GetBundleRange(*topicName) + req.NoError(err) + req.NotEmpty(bundleRange) + + clientURL := clientEndpointFunc(lookupResult) + pulsarClient, err := NewClient(ClientOptions{URL: clientURL}) + req.NoError(err) + defer pulsarClient.Close() + + producer, err := pulsarClient.CreateProducer(ProducerOptions{ + Topic: topicName.String(), + }) + req.NoError(err) + defer producer.Close() + + consumer, err := pulsarClient.Subscribe(ConsumerOptions{ + Topic: topicName.String(), + SubscriptionName: fmt.Sprintf("my-sub-%v", time.Now().Nanosecond()), + SubscriptionInitialPosition: SubscriptionPositionEarliest, + }) + req.NoError(err) + defer consumer.Close() + + pulsarClientImpl := pulsarClient.(*client) + lookupRequestCounterMock := mockCounter{} + pulsarClientImpl.metrics.LookupRequestsCount = &lookupRequestCounterMock + + messageCountBeforeUnload := 100 + messageCountDuringUnload := 100 + messageCountAfterUnload := 100 + messageCount := messageCountBeforeUnload + messageCountDuringUnload + messageCountAfterUnload + + // Signals all goroutines have completed + wgRoutines := sync.WaitGroup{} + wgRoutines.Add(2) + + // Signals unload has completed + wgUnload := sync.WaitGroup{} + wgUnload.Add(1) + + // Signals both producer and consumer have processed `messageCountBeforeUnload` messages + wgSendAndReceiveMessages := sync.WaitGroup{} + wgSendAndReceiveMessages.Add(2) + + // Producer + go func() { + defer wgRoutines.Done() + + for i := 0; i < messageCount; i++ { + if i == messageCountBeforeUnload+messageCountDuringUnload { + wgUnload.Wait() + } + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + defer cancel() + + pm := ProducerMessage{Payload: []byte(fmt.Sprintf("hello-%d", i))} + _, err := producer.Send(ctx, &pm) + req.NoError(err) + req.NoError(ctx.Err()) + + if i == messageCountBeforeUnload { + wgSendAndReceiveMessages.Done() + } + } + }() + + // Consumer + go func() { + defer wgRoutines.Done() + + for i := 0; i < messageCount; i++ { + if i == messageCountBeforeUnload+messageCountDuringUnload { + wgUnload.Wait() + } + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + defer cancel() + + _, err := consumer.Receive(ctx) + req.NoError(err) + req.NoError(ctx.Err()) + + if i == messageCountBeforeUnload { + wgSendAndReceiveMessages.Done() + } + } + }() + + // Unload the bundle, triggering the producers and consumers to reconnect to the specified broker. + wgSendAndReceiveMessages.Wait() + unloadEndpoint := unloadEndpointFunc(lookupResult) + unloadURL := fmt.Sprintf( + "/admin/v2/namespaces/%s/%s/%s/unload?destinationBroker=%s", tenant, namespace, bundleRange, dstTopicBrokerURL) + makeHTTPCall(t, http.MethodPut, unloadEndpoint+unloadURL, "") + wgUnload.Done() + + wgRoutines.Wait() + req.Equal(int32(0), lookupRequestCounterMock.count.Load()) +} diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index cb8c680d48..9244328bc2 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -69,7 +69,7 @@ type ConnectionListener interface { ReceivedSendReceipt(response *pb.CommandSendReceipt) // ConnectionClosed close the TCP connection. - ConnectionClosed() + ConnectionClosed(closeProducer *pb.CommandCloseProducer) } // Connection is a interface of client cnx. @@ -84,6 +84,7 @@ type Connection interface { ID() string GetMaxMessageSize() int32 Close() + IsProxied() bool } type ConsumerHandler interface { @@ -92,7 +93,7 @@ type ConsumerHandler interface { ActiveConsumerChanged(isActive bool) // ConnectionClosed close the TCP connection. - ConnectionClosed() + ConnectionClosed(closeConsumer *pb.CommandCloseConsumer) } type connectionState int32 @@ -312,7 +313,7 @@ func (c *connection) doHandshake() bool { }, } - if c.logicalAddr.Host != c.physicalAddr.Host { + if c.IsProxied() { cmdConnect.ProxyToBrokerUrl = proto.String(c.logicalAddr.Host) } c.writeCommand(baseCommand(pb.BaseCommand_CONNECT, cmdConnect)) @@ -343,6 +344,10 @@ func (c *connection) doHandshake() bool { return true } +func (c *connection) IsProxied() bool { + return c.logicalAddr.Host != c.physicalAddr.Host +} + func (c *connection) waitUntilReady() error { // If we are going to call cond.Wait() at all, then we must call it _before_ we call cond.Broadcast(). // The lock is held here to prevent changeState() from calling cond.Broadcast() in the time between @@ -893,7 +898,7 @@ func (c *connection) handleCloseConsumer(closeConsumer *pb.CommandCloseConsumer) c.log.Infof("Broker notification of Closed consumer: %d", consumerID) if consumer, ok := c.consumerHandler(consumerID); ok { - consumer.ConnectionClosed() + consumer.ConnectionClosed(closeConsumer) c.DeleteConsumeHandler(consumerID) } else { c.log.WithField("consumerID", consumerID).Warnf("Consumer with ID not found while closing consumer") @@ -917,7 +922,7 @@ func (c *connection) handleCloseProducer(closeProducer *pb.CommandCloseProducer) producer, ok := c.deletePendingProducers(producerID) // did we find a producer? if ok { - producer.ConnectionClosed() + producer.ConnectionClosed(closeProducer) } else { c.log.WithField("producerID", producerID).Warn("Producer with ID not found while closing producer") } @@ -1024,12 +1029,12 @@ func (c *connection) Close() { // notify producers connection closed for _, listener := range listeners { - listener.ConnectionClosed() + listener.ConnectionClosed(nil) } // notify consumers connection closed for _, handler := range consumerHandlers { - handler.ConnectionClosed() + handler.ConnectionClosed(nil) } c.metrics.ConnectionsClosed.Inc() diff --git a/pulsar/internal/lookup_service.go b/pulsar/internal/lookup_service.go index e30bec1d73..8b7fb01b2f 100644 --- a/pulsar/internal/lookup_service.go +++ b/pulsar/internal/lookup_service.go @@ -64,6 +64,8 @@ type LookupService interface { // GetSchema returns schema for a given version. GetSchema(topic string, schemaVersion []byte) (schema *pb.Schema, err error) + GetBrokerAddress(brokerServiceURL string, proxyThroughServiceURL bool) (*LookupResult, error) + // Closable Allow Lookup Service's internal client to be able to closed Closable } @@ -107,29 +109,26 @@ func (ls *lookupService) GetSchema(topic string, schemaVersion []byte) (schema * return res.Response.GetSchemaResponse.Schema, nil } -func (ls *lookupService) getBrokerAddress(lr *pb.CommandLookupTopicResponse) (logicalAddress *url.URL, - physicalAddress *url.URL, err error) { - if ls.tlsEnabled { - logicalAddress, err = url.ParseRequestURI(lr.GetBrokerServiceUrlTls()) - } else { - logicalAddress, err = url.ParseRequestURI(lr.GetBrokerServiceUrl()) - } - +func (ls *lookupService) GetBrokerAddress(brokerServiceURL string, proxyThroughServiceURL bool) (*LookupResult, error) { + logicalAddress, err := url.ParseRequestURI(brokerServiceURL) if err != nil { - return nil, nil, err + return nil, err } - var physicalAddr *url.URL - if lr.GetProxyThroughServiceUrl() { - physicalAddr, err = ls.serviceNameResolver.ResolveHost() + var physicalAddress *url.URL + if proxyThroughServiceURL { + physicalAddress, err = ls.serviceNameResolver.ResolveHost() if err != nil { - return nil, nil, err + return nil, err } } else { - physicalAddr = logicalAddress + physicalAddress = logicalAddress } - return logicalAddress, physicalAddr, nil + return &LookupResult{ + LogicalAddr: logicalAddress, + PhysicalAddr: physicalAddress, + }, nil } // Follow brokers redirect up to certain number of times @@ -154,7 +153,8 @@ func (ls *lookupService) Lookup(topic string) (*LookupResult, error) { switch *lr.Response { case pb.CommandLookupTopicResponse_Redirect: - logicalAddress, physicalAddr, err := ls.getBrokerAddress(lr) + brokerServiceURL := selectServiceURL(ls.tlsEnabled, lr.GetBrokerServiceUrl(), lr.GetBrokerServiceUrlTls()) + lookupResult, err := ls.GetBrokerAddress(brokerServiceURL, lr.GetProxyThroughServiceUrl()) if err != nil { return nil, err } @@ -163,12 +163,13 @@ func (ls *lookupService) Lookup(topic string) (*LookupResult, error) { topic, lr.BrokerServiceUrl, lr.BrokerServiceUrlTls, lr.ProxyThroughServiceUrl) id := ls.rpcClient.NewRequestID() - res, err = ls.rpcClient.Request(logicalAddress, physicalAddr, id, pb.BaseCommand_LOOKUP, &pb.CommandLookupTopic{ - RequestId: &id, - Topic: &topic, - Authoritative: lr.Authoritative, - AdvertisedListenerName: proto.String(ls.listenerName), - }) + res, err = ls.rpcClient.Request(lookupResult.LogicalAddr, lookupResult.PhysicalAddr, id, pb.BaseCommand_LOOKUP, + &pb.CommandLookupTopic{ + RequestId: &id, + Topic: &topic, + Authoritative: lr.Authoritative, + AdvertisedListenerName: proto.String(ls.listenerName), + }) if err != nil { return nil, err } @@ -180,16 +181,8 @@ func (ls *lookupService) Lookup(topic string) (*LookupResult, error) { ls.log.Debugf("Successfully looked up topic{%s} on broker. %s / %s - Use proxy: %t", topic, lr.GetBrokerServiceUrl(), lr.GetBrokerServiceUrlTls(), lr.GetProxyThroughServiceUrl()) - logicalAddress, physicalAddress, err := ls.getBrokerAddress(lr) - if err != nil { - return nil, err - } - - return &LookupResult{ - LogicalAddr: logicalAddress, - PhysicalAddr: physicalAddress, - }, nil - + brokerServiceURL := selectServiceURL(ls.tlsEnabled, lr.GetBrokerServiceUrl(), lr.GetBrokerServiceUrlTls()) + return ls.GetBrokerAddress(brokerServiceURL, lr.GetProxyThroughServiceUrl()) case pb.CommandLookupTopicResponse_Failed: ls.log.WithFields(log.Fields{ "topic": topic, @@ -284,19 +277,15 @@ type httpLookupService struct { metrics *Metrics } -func (h *httpLookupService) getBrokerAddress(ld *httpLookupData) (logicalAddress *url.URL, - physicalAddress *url.URL, err error) { - if h.tlsEnabled { - logicalAddress, err = url.ParseRequestURI(ld.BrokerURLTLS) - } else { - logicalAddress, err = url.ParseRequestURI(ld.BrokerURL) - } - +func (h *httpLookupService) GetBrokerAddress(brokerServiceURL string, _ bool) (*LookupResult, error) { + logicalAddress, err := url.ParseRequestURI(brokerServiceURL) if err != nil { - return nil, nil, err + return nil, err } - - return logicalAddress, logicalAddress, nil + return &LookupResult{ + LogicalAddr: logicalAddress, + PhysicalAddr: logicalAddress, + }, err } func (h *httpLookupService) Lookup(topic string) (*LookupResult, error) { @@ -319,16 +308,8 @@ func (h *httpLookupService) Lookup(topic string) (*LookupResult, error) { h.log.Debugf("Successfully looked up topic{%s} on http broker. %+v", topic, lookupData) - logicalAddress, physicalAddress, err := h.getBrokerAddress(lookupData) - if err != nil { - return nil, err - } - - return &LookupResult{ - LogicalAddr: logicalAddress, - PhysicalAddr: physicalAddress, - }, nil - + brokerServiceURL := selectServiceURL(h.tlsEnabled, lookupData.BrokerURL, lookupData.BrokerURLTLS) + return h.GetBrokerAddress(brokerServiceURL, false /* ignored */) } func (h *httpLookupService) GetPartitionedTopicMetadata(topic string) (*PartitionedTopicMetadata, @@ -397,3 +378,10 @@ func NewHTTPLookupService(httpClient HTTPClient, serviceURL *url.URL, serviceNam metrics: metrics, } } + +func selectServiceURL(tlsEnabled bool, brokerServiceURL, brokerServiceURLTLS string) string { + if tlsEnabled { + return brokerServiceURLTLS + } + return brokerServiceURL +} diff --git a/pulsar/internal/pulsar_proto/PulsarApi.pb.go b/pulsar/internal/pulsar_proto/PulsarApi.pb.go index c8e6ad9b91..5c6e0f3ef5 100644 --- a/pulsar/internal/pulsar_proto/PulsarApi.pb.go +++ b/pulsar/internal/pulsar_proto/PulsarApi.pb.go @@ -1,4 +1,4 @@ -//* +// // 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 @@ -19,7 +19,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.28.1 -// protoc v3.21.9 +// protoc v3.21.12 // source: PulsarApi.proto package proto @@ -1744,7 +1744,7 @@ type MessageMetadata struct { // Property set on replicated message, // includes the source cluster name ReplicatedFrom *string `protobuf:"bytes,5,opt,name=replicated_from,json=replicatedFrom" json:"replicated_from,omitempty"` - //key to decide partition for the msg + // key to decide partition for the msg 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=replicateTo" json:"replicate_to,omitempty"` @@ -1752,7 +1752,7 @@ type MessageMetadata struct { 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; + // optional sfixed64 checksum = 10; // differentiate single and batch message metadata 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. @@ -1777,7 +1777,7 @@ type MessageMetadata struct { // transaction related message info 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 + // / Add highest sequence id to support batch message with external sequence id 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"` @@ -2786,7 +2786,7 @@ type CommandSubscribe struct { // 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=startMessageId" json:"start_message_id,omitempty"` - /// Add optional metadata key=value to this consumer + // / 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=readCompacted" json:"read_compacted,omitempty"` Schema *Schema `protobuf:"bytes,12,opt,name=schema" json:"schema,omitempty"` @@ -3164,7 +3164,6 @@ type CommandLookupTopic struct { // to the proxy. 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"` } @@ -3366,8 +3365,8 @@ func (x *CommandLookupTopicResponse) GetProxyThroughServiceUrl() bool { return Default_CommandLookupTopicResponse_ProxyThroughServiceUrl } -/// Create a new Producer on a topic, assigning the given producer_id, -/// all messages sent with this producer_id will be persisted on the topic +// / Create a new Producer on a topic, assigning the given producer_id, +// / all messages sent with this producer_id will be persisted on the topic type CommandProducer struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3376,11 +3375,11 @@ type CommandProducer struct { Topic *string `protobuf:"bytes,1,req,name=topic" json:"topic,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 + // / 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=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 + // / Add optional metadata key=value to this producer Metadata []*KeyValue `protobuf:"bytes,6,rep,name=metadata" json:"metadata,omitempty"` Schema *Schema `protobuf:"bytes,7,opt,name=schema" json:"schema,omitempty"` // If producer reconnect to broker, the epoch of this producer will +1 @@ -3546,7 +3545,7 @@ type CommandSend struct { 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 + // / Add highest sequence id to support batch message with external sequence id 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"` // Specify if the message being published is a Pulsar marker or not @@ -4394,8 +4393,10 @@ type CommandCloseProducer struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - 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"` + 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"` + AssignedBrokerServiceUrl *string `protobuf:"bytes,3,opt,name=assignedBrokerServiceUrl" json:"assignedBrokerServiceUrl,omitempty"` + AssignedBrokerServiceUrlTls *string `protobuf:"bytes,4,opt,name=assignedBrokerServiceUrlTls" json:"assignedBrokerServiceUrlTls,omitempty"` } func (x *CommandCloseProducer) Reset() { @@ -4444,13 +4445,29 @@ func (x *CommandCloseProducer) GetRequestId() uint64 { return 0 } +func (x *CommandCloseProducer) GetAssignedBrokerServiceUrl() string { + if x != nil && x.AssignedBrokerServiceUrl != nil { + return *x.AssignedBrokerServiceUrl + } + return "" +} + +func (x *CommandCloseProducer) GetAssignedBrokerServiceUrlTls() string { + if x != nil && x.AssignedBrokerServiceUrlTls != nil { + return *x.AssignedBrokerServiceUrlTls + } + return "" +} + type CommandCloseConsumer struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - 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"` + 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"` + AssignedBrokerServiceUrl *string `protobuf:"bytes,3,opt,name=assignedBrokerServiceUrl" json:"assignedBrokerServiceUrl,omitempty"` + AssignedBrokerServiceUrlTls *string `protobuf:"bytes,4,opt,name=assignedBrokerServiceUrlTls" json:"assignedBrokerServiceUrlTls,omitempty"` } func (x *CommandCloseConsumer) Reset() { @@ -4499,6 +4516,20 @@ func (x *CommandCloseConsumer) GetRequestId() uint64 { return 0 } +func (x *CommandCloseConsumer) GetAssignedBrokerServiceUrl() string { + if x != nil && x.AssignedBrokerServiceUrl != nil { + return *x.AssignedBrokerServiceUrl + } + return "" +} + +func (x *CommandCloseConsumer) GetAssignedBrokerServiceUrlTls() string { + if x != nil && x.AssignedBrokerServiceUrlTls != nil { + return *x.AssignedBrokerServiceUrlTls + } + return "" +} + type CommandRedeliverUnacknowledgedMessages struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -4617,7 +4648,7 @@ func (x *CommandSuccess) GetSchema() *Schema { return nil } -/// Response from CommandProducer +// / Response from CommandProducer type CommandProducerSuccess struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -4925,31 +4956,31 @@ type CommandConsumerStatsResponse struct { 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 + // / 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 + // / Total throughput delivered to the consumer. bytes/s MsgThroughputOut *float64 `protobuf:"fixed64,5,opt,name=msgThroughputOut" json:"msgThroughputOut,omitempty"` - /// Total rate of messages redelivered by this consumer. msg/s + // / Total rate of messages redelivered by this consumer. msg/s MsgRateRedeliver *float64 `protobuf:"fixed64,6,opt,name=msgRateRedeliver" json:"msgRateRedeliver,omitempty"` - /// Name of the consumer + // / Name of the consumer ConsumerName *string `protobuf:"bytes,7,opt,name=consumerName" json:"consumerName,omitempty"` - /// Number of available message permits for the consumer + // / Number of available message permits for the consumer AvailablePermits *uint64 `protobuf:"varint,8,opt,name=availablePermits" json:"availablePermits,omitempty"` - /// Number of unacknowledged messages for the consumer + // / Number of unacknowledged messages for the consumer UnackedMessages *uint64 `protobuf:"varint,9,opt,name=unackedMessages" json:"unackedMessages,omitempty"` - /// Flag to verify if consumer is blocked due to reaching threshold of unacked messages + // / Flag to verify if consumer is blocked due to reaching threshold of unacked messages BlockedConsumerOnUnackedMsgs *bool `protobuf:"varint,10,opt,name=blockedConsumerOnUnackedMsgs" json:"blockedConsumerOnUnackedMsgs,omitempty"` - /// Address of this consumer + // / Address of this consumer Address *string `protobuf:"bytes,11,opt,name=address" json:"address,omitempty"` - /// Timestamp of connection + // / Timestamp of connection ConnectedSince *string `protobuf:"bytes,12,opt,name=connectedSince" json:"connectedSince,omitempty"` - /// Whether this subscription is Exclusive or Shared or Failover + // / Whether this subscription is Exclusive or Shared or Failover Type *string `protobuf:"bytes,13,opt,name=type" json:"type,omitempty"` - /// Total rate of messages expired on this subscription. msg/s + // / Total rate of messages expired on this subscription. msg/s MsgRateExpired *float64 `protobuf:"fixed64,14,opt,name=msgRateExpired" json:"msgRateExpired,omitempty"` - /// Number of messages in the subscription backlog + // / Number of messages in the subscription backlog MsgBacklog *uint64 `protobuf:"varint,15,opt,name=msgBacklog" json:"msgBacklog,omitempty"` - /// Total rate of messages ack. msg/s + // / Total rate of messages ack. msg/s MessageAckRate *float64 `protobuf:"fixed64,16,opt,name=messageAckRate" json:"messageAckRate,omitempty"` } @@ -8235,345 +8266,394 @@ var file_PulsarApi_proto_rawDesc = []byte{ 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0x56, 0x0a, 0x14, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, - 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, - 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, - 0x64, 0x22, 0x56, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0xd4, 0x01, 0x0a, 0x14, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, + 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, + 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x12, 0x40, + 0x0a, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, 0x73, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, + 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, 0x73, + 0x22, 0xd4, 0x01, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x22, 0xae, 0x01, 0x0a, 0x26, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, 0x6e, - 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x3c, 0x0a, 0x0b, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x49, 0x64, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, - 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x63, 0x6f, 0x6e, - 0x73, 0x75, 0x6d, 0x65, 0x72, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x22, 0x5d, 0x0a, 0x0e, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, - 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2c, 0x0a, 0x06, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xff, 0x01, 0x0a, 0x16, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x53, 0x75, 0x63, - 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x72, 0x6f, 0x64, - 0x75, 0x63, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x10, 0x6c, 0x61, 0x73, 0x74, - 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x03, 0x3a, 0x02, 0x2d, 0x31, 0x52, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x65, 0x71, 0x75, - 0x65, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, - 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, - 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x2b, - 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x0d, 0x70, 0x72, - 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x61, 0x64, 0x79, 0x22, 0x78, 0x0a, 0x0c, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1d, 0x0a, 0x0a, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, - 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x02, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, - 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x50, 0x69, 0x6e, 0x67, 0x22, 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, - 0x6f, 0x6e, 0x67, 0x22, 0x56, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, - 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, - 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, - 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x02, 0x28, 0x04, 0x52, - 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0x98, 0x05, 0x0a, 0x1c, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, - 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, - 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, - 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x6d, 0x73, - 0x67, 0x52, 0x61, 0x74, 0x65, 0x4f, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, - 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, - 0x67, 0x54, 0x68, 0x72, 0x6f, 0x75, 0x67, 0x68, 0x70, 0x75, 0x74, 0x4f, 0x75, 0x74, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x01, 0x52, 0x10, 0x6d, 0x73, 0x67, 0x54, 0x68, 0x72, 0x6f, 0x75, 0x67, 0x68, - 0x70, 0x75, 0x74, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, - 0x65, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, - 0x52, 0x10, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, - 0x65, 0x72, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x61, - 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, - 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x10, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, - 0x62, 0x6c, 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x10, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, - 0x74, 0x73, 0x12, 0x28, 0x0a, 0x0f, 0x75, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x75, 0x6e, 0x61, - 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x42, 0x0a, 0x1c, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4f, - 0x6e, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x1c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x4f, 0x6e, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, - 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x53, 0x69, 0x6e, 0x63, 0x65, 0x18, 0x0c, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x53, 0x69, 0x6e, - 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, - 0x65, 0x45, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, - 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x45, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x12, 0x1e, - 0x0a, 0x0a, 0x6d, 0x73, 0x67, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x18, 0x0f, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x0a, 0x6d, 0x73, 0x67, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x12, 0x26, - 0x0a, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x41, 0x63, 0x6b, 0x52, 0x61, 0x74, 0x65, - 0x18, 0x10, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x41, - 0x63, 0x6b, 0x52, 0x61, 0x74, 0x65, 0x22, 0x59, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, - 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, - 0x64, 0x22, 0xe5, 0x01, 0x0a, 0x1f, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, - 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x43, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x0b, 0x32, 0x1b, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x61, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x55, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x61, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x55, 0x72, 0x6c, 0x12, 0x40, 0x0a, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, + 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, + 0x6c, 0x54, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x61, 0x73, 0x73, 0x69, + 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, 0x73, 0x22, 0xae, 0x01, 0x0a, 0x26, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, + 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x49, 0x64, 0x12, 0x3c, 0x0a, 0x0b, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, + 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, + 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, + 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x65, 0x70, + 0x6f, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x22, 0x5d, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, + 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xff, 0x01, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x53, 0x75, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, + 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, + 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, + 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x3a, 0x02, 0x2d, 0x31, 0x52, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, + 0x63, 0x65, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x2b, 0x0a, 0x0e, + 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x64, + 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x61, 0x64, 0x79, 0x22, 0x78, 0x0a, 0x0c, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x18, 0x02, 0x20, 0x02, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, + 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x69, + 0x6e, 0x67, 0x22, 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x6f, 0x6e, + 0x67, 0x22, 0x56, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0x98, 0x05, 0x0a, 0x1c, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, + 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, + 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x6d, 0x73, 0x67, 0x52, + 0x61, 0x74, 0x65, 0x4f, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, 0x6d, 0x73, + 0x67, 0x52, 0x61, 0x74, 0x65, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, 0x67, 0x54, + 0x68, 0x72, 0x6f, 0x75, 0x67, 0x68, 0x70, 0x75, 0x74, 0x4f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x01, 0x52, 0x10, 0x6d, 0x73, 0x67, 0x54, 0x68, 0x72, 0x6f, 0x75, 0x67, 0x68, 0x70, 0x75, + 0x74, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x52, + 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, 0x52, 0x10, + 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, + 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x10, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, + 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x10, + 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, + 0x12, 0x28, 0x0a, 0x0f, 0x75, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x75, 0x6e, 0x61, 0x63, 0x6b, + 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x42, 0x0a, 0x1c, 0x62, 0x6c, + 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4f, 0x6e, 0x55, + 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x1c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x4f, 0x6e, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, 0x12, 0x18, + 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x65, 0x64, 0x53, 0x69, 0x6e, 0x63, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x53, 0x69, 0x6e, 0x63, 0x65, + 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x74, 0x79, 0x70, 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x45, + 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, 0x6d, 0x73, + 0x67, 0x52, 0x61, 0x74, 0x65, 0x45, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x12, 0x1e, 0x0a, 0x0a, + 0x6d, 0x73, 0x67, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x0a, 0x6d, 0x73, 0x67, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x12, 0x26, 0x0a, 0x0e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x41, 0x63, 0x6b, 0x52, 0x61, 0x74, 0x65, 0x18, 0x10, + 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x41, 0x63, 0x6b, + 0x52, 0x61, 0x74, 0x65, 0x22, 0x59, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, + 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, + 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, + 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x22, + 0xe5, 0x01, 0x0a, 0x1f, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, + 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x43, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, + 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x5e, 0x0a, 0x1d, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, + 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0d, 0x6c, 0x61, 0x73, - 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x5e, 0x0a, 0x1d, 0x63, 0x6f, 0x6e, - 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x1a, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4d, 0x61, 0x72, 0x6b, 0x44, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xa7, 0x02, 0x0a, 0x1b, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, - 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x1a, 0x63, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4d, 0x61, 0x72, 0x6b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xa7, 0x02, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x4e, 0x0a, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x4d, 0x6f, + 0x64, 0x65, 0x3a, 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x52, 0x04, + 0x6d, 0x6f, 0x64, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x70, + 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x33, 0x0a, 0x04, + 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, + 0x4e, 0x54, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x4e, 0x4f, 0x4e, 0x5f, 0x50, 0x45, 0x52, 0x53, + 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4c, 0x4c, 0x10, + 0x02, 0x22, 0xc0, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, - 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x4e, 0x0a, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, - 0x4d, 0x6f, 0x64, 0x65, 0x3a, 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, - 0x52, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, - 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x1f, 0x0a, - 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x33, - 0x0a, 0x04, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, - 0x54, 0x45, 0x4e, 0x54, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x4e, 0x4f, 0x4e, 0x5f, 0x50, 0x45, - 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4c, - 0x4c, 0x10, 0x02, 0x22, 0xc0, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, - 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, - 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x73, 0x12, 0x21, 0x0a, 0x08, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x08, 0x66, 0x69, 0x6c, - 0x74, 0x65, 0x72, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, - 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x12, 0x1e, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, - 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x07, 0x63, - 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x22, 0xbb, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x6f, 0x70, 0x69, + 0x63, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, + 0x12, 0x21, 0x0a, 0x08, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, + 0x73, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, + 0x48, 0x61, 0x73, 0x68, 0x12, 0x1e, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x07, 0x63, 0x68, 0x61, + 0x6e, 0x67, 0x65, 0x64, 0x22, 0xbb, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1d, + 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, + 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, + 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, + 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, 0x09, 0x52, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x02, + 0x28, 0x09, 0x52, 0x0d, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, + 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, + 0x73, 0x68, 0x22, 0x93, 0x01, 0x0a, 0x1c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, + 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, + 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, + 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x9f, 0x01, 0x0a, 0x17, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, + 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x65, 0x77, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, + 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0a, + 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x5a, 0x0a, 0x1a, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, + 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, + 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, + 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x22, 0x6e, 0x0a, 0x10, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, + 0x69, 0x63, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, + 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xed, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, + 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, + 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x7d, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, + 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, + 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x03, 0x20, 0x02, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xc7, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, + 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, + 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, + 0x56, 0x0a, 0x1d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, - 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, - 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1c, - 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, - 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x04, - 0x20, 0x02, 0x28, 0x09, 0x52, 0x0d, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x50, 0x61, 0x74, 0x74, - 0x65, 0x72, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, - 0x73, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, - 0x48, 0x61, 0x73, 0x68, 0x22, 0x93, 0x01, 0x0a, 0x1c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, - 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, - 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, - 0x72, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0a, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x9f, 0x01, 0x0a, 0x17, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, - 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, - 0x68, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x65, 0x77, 0x54, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x65, - 0x6c, 0x65, 0x74, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, - 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x5a, 0x0a, 0x1a, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, - 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x16, 0x0a, 0x05, 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x3a, 0x01, + 0x30, 0x52, 0x04, 0x74, 0x63, 0x49, 0x64, 0x22, 0x8a, 0x01, 0x0a, 0x1e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, - 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, - 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x22, 0x6e, 0x0a, 0x10, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1d, 0x0a, 0x0a, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, - 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, - 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xed, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6f, - 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, - 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, - 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, - 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x7d, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x02, - 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x02, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, - 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xc7, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, - 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, - 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, - 0x6e, 0x22, 0x56, 0x0a, 0x1d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, - 0x64, 0x12, 0x16, 0x0a, 0x05, 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, - 0x3a, 0x01, 0x30, 0x52, 0x04, 0x74, 0x63, 0x49, 0x64, 0x22, 0x8a, 0x01, 0x0a, 0x1e, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, + 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4e, + 0x65, 0x77, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x5f, 0x74, 0x74, 0x6c, 0x5f, + 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, + 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x54, 0x74, 0x6c, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, + 0x16, 0x0a, 0x05, 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, + 0x30, 0x52, 0x04, 0x74, 0x63, 0x49, 0x64, 0x22, 0xd9, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, + 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, + 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, + 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, + 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, + 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0xb1, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, + 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, + 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, + 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, + 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, + 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, + 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, + 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xe4, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, + 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, - 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, + 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, + 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, + 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, + 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, + 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x48, + 0x0a, 0x0c, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x14, + 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x02, 0x28, 0x09, 0x52, 0x05, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x12, 0x22, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd4, 0x01, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, + 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, + 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, + 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, + 0x3e, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, + 0xe7, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, + 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, + 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, + 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, + 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, + 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, + 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, + 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, + 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x0d, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, + 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, + 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, + 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, + 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, + 0x74, 0x73, 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x09, 0x74, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd9, 0x01, 0x0a, 0x15, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, + 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, + 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, + 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, + 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, + 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x5f, 0x74, 0x74, - 0x6c, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, - 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x54, 0x74, 0x6c, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, - 0x73, 0x12, 0x16, 0x0a, 0x05, 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, - 0x3a, 0x01, 0x30, 0x52, 0x04, 0x74, 0x63, 0x49, 0x64, 0x22, 0xd9, 0x01, 0x0a, 0x15, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xa8, 0x02, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, - 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xb1, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, - 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, - 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, - 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, - 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, - 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, - 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, - 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xe4, 0x01, 0x0a, 0x20, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, - 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, - 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, - 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, - 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, - 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, - 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, - 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, - 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x22, 0x48, 0x0a, 0x0c, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x02, 0x28, 0x09, 0x52, - 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x22, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd4, 0x01, 0x0a, 0x1b, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, - 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, - 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, - 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, - 0x73, 0x12, 0x3e, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x22, 0xe7, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, - 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, - 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, - 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, - 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, - 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, - 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, - 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x0d, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, - 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, - 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, - 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, - 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, - 0x42, 0x69, 0x74, 0x73, 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x09, 0x74, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd9, 0x01, 0x0a, - 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, + 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x74, + 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, 0x6e, 0x69, + 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, 0x66, 0x5f, + 0x6c, 0x6f, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, + 0x69, 0x74, 0x73, 0x4f, 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, + 0x6b, 0x22, 0xe4, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, + 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, @@ -8586,27 +8666,30 @@ var file_PulsarApi_proto_rawDesc = []byte{ 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xa8, 0x02, 0x0a, 0x18, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, - 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, - 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, - 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, - 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, - 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x14, 0x0a, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x09, 0x74, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, - 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, - 0x66, 0x5f, 0x6c, 0x6f, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, - 0x74, 0x42, 0x69, 0x74, 0x73, 0x4f, 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, - 0x61, 0x72, 0x6b, 0x22, 0xe4, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, - 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xd5, 0x02, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, + 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, + 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, + 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, + 0x3e, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x74, 0x78, + 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, 0x66, 0x5f, 0x6c, + 0x6f, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, + 0x74, 0x73, 0x4f, 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, + 0x22, 0xe7, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, + 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, @@ -8619,489 +8702,452 @@ var file_PulsarApi_proto_rawDesc = []byte{ 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xd5, 0x02, 0x0a, 0x1b, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, - 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, - 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, - 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, - 0x73, 0x12, 0x3e, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, - 0x74, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, 0x66, - 0x5f, 0x6c, 0x6f, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x04, 0x52, 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, - 0x42, 0x69, 0x74, 0x73, 0x4f, 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, - 0x72, 0x6b, 0x22, 0xe7, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, - 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, - 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, - 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, - 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, - 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, - 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x9a, 0x2e, 0x0a, - 0x0b, 0x42, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x32, 0x0a, 0x04, - 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x02, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x42, 0x61, 0x73, 0x65, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, - 0x12, 0x36, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, - 0x07, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x12, 0x3c, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, + 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x9a, 0x2e, 0x0a, 0x0b, 0x42, + 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x32, 0x0a, 0x04, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x01, 0x20, 0x02, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x42, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x36, + 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x07, 0x63, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x12, 0x3c, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x65, 0x64, 0x12, 0x3c, 0x0a, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x12, 0x39, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, + 0x63, 0x65, 0x72, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, 0x2d, 0x0a, + 0x04, 0x73, 0x65, 0x6e, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x09, 0x63, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x12, 0x3c, 0x0a, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x62, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x04, 0x73, 0x65, 0x6e, 0x64, 0x12, 0x43, 0x0a, 0x0c, + 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x63, + 0x65, 0x69, 0x70, 0x74, 0x52, 0x0b, 0x73, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x63, 0x65, 0x69, 0x70, + 0x74, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, + 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x73, 0x65, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x12, 0x36, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, + 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2a, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x63, 0x6b, 0x52, + 0x03, 0x61, 0x63, 0x6b, 0x12, 0x2d, 0x0a, 0x04, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x0b, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x04, 0x66, + 0x6c, 0x6f, 0x77, 0x12, 0x42, 0x0a, 0x0b, 0x75, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x55, + 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x0b, 0x75, 0x6e, 0x73, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x36, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, + 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, + 0x30, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x12, 0x49, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x64, 0x75, + 0x63, 0x65, 0x72, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x62, 0x65, 0x12, 0x39, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, - 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, - 0x2d, 0x0a, 0x04, 0x73, 0x65, 0x6e, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, + 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x0d, 0x63, + 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x0e, + 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x18, 0x10, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, + 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x43, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x12, 0x4f, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x64, 0x75, + 0x63, 0x65, 0x72, 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, + 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, + 0x72, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x69, 0x6e, 0x67, + 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x69, 0x6e, + 0x67, 0x52, 0x04, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x6f, 0x6e, 0x67, 0x18, + 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x6f, 0x6e, 0x67, + 0x52, 0x04, 0x70, 0x6f, 0x6e, 0x67, 0x12, 0x7e, 0x0a, 0x1f, 0x72, 0x65, 0x64, 0x65, 0x6c, 0x69, + 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, + 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x34, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, + 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x1f, 0x72, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, + 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x5b, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x15, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x52, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x12, 0x73, 0x0a, 0x19, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x6c, 0x6f, 0x6f, 0x6b, + 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x04, 0x73, 0x65, 0x6e, 0x64, 0x12, 0x43, - 0x0a, 0x0c, 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x18, 0x07, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x52, - 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x52, 0x0b, 0x73, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x63, 0x65, - 0x69, 0x70, 0x74, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, - 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x73, 0x65, 0x6e, 0x64, 0x45, 0x72, 0x72, - 0x6f, 0x72, 0x12, 0x36, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x09, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2a, 0x0a, 0x03, 0x61, 0x63, - 0x6b, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x63, - 0x6b, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x12, 0x2d, 0x0a, 0x04, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x0b, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x46, 0x6c, 0x6f, 0x77, 0x52, - 0x04, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x42, 0x0a, 0x0b, 0x75, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x62, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x0b, 0x75, 0x6e, - 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x36, 0x0a, 0x07, 0x73, 0x75, 0x63, - 0x63, 0x65, 0x73, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, - 0x73, 0x12, 0x30, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x12, 0x49, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x70, 0x72, 0x6f, - 0x64, 0x75, 0x63, 0x65, 0x72, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, - 0x0d, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, 0x49, - 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, - 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, - 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x12, 0x4f, 0x0a, 0x10, 0x70, 0x72, 0x6f, - 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x11, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, - 0x65, 0x72, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x65, 0x72, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x69, - 0x6e, 0x67, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, - 0x69, 0x6e, 0x67, 0x52, 0x04, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x6f, 0x6e, - 0x67, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x6f, - 0x6e, 0x67, 0x52, 0x04, 0x70, 0x6f, 0x6e, 0x67, 0x12, 0x7e, 0x0a, 0x1f, 0x72, 0x65, 0x64, 0x65, - 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, - 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x14, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x34, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, - 0x72, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x1f, 0x72, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, - 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x5b, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x15, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x52, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, - 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x73, 0x0a, 0x19, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, - 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, - 0x19, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x6c, 0x6f, - 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, - 0x63, 0x52, 0x0b, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x5a, - 0x0a, 0x13, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x13, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0d, 0x63, 0x6f, - 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x18, 0x19, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, - 0x74, 0x61, 0x74, 0x73, 0x12, 0x60, 0x0a, 0x15, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1a, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, - 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, - 0x15, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, 0x65, - 0x64, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x1b, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, - 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, - 0x65, 0x64, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2d, 0x0a, 0x04, - 0x73, 0x65, 0x65, 0x6b, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x04, 0x73, 0x65, 0x65, 0x6b, 0x12, 0x51, 0x0a, 0x10, 0x67, - 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x18, - 0x1d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, - 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x10, 0x67, 0x65, - 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x69, - 0x0a, 0x18, 0x67, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x2d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, - 0x18, 0x67, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, - 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5f, 0x0a, 0x16, 0x61, 0x63, 0x74, - 0x69, 0x76, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x68, 0x61, - 0x6e, 0x67, 0x65, 0x18, 0x1f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x6d, 0x61, 0x6e, 0x64, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, + 0x0b, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x5a, 0x0a, 0x13, + 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x43, 0x68, - 0x61, 0x6e, 0x67, 0x65, 0x52, 0x14, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x67, 0x65, - 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x18, 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, - 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x52, 0x14, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, - 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x75, 0x0a, 0x1c, 0x67, 0x65, 0x74, + 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x52, 0x13, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0d, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x12, 0x60, 0x0a, 0x15, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1a, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x15, 0x63, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, + 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, 0x6e, 0x64, + 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, + 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2d, 0x0a, 0x04, 0x73, 0x65, + 0x65, 0x6b, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, + 0x65, 0x65, 0x6b, 0x52, 0x04, 0x73, 0x65, 0x65, 0x6b, 0x12, 0x51, 0x0a, 0x10, 0x67, 0x65, 0x74, + 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x18, 0x1d, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, + 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x10, 0x67, 0x65, 0x74, 0x4c, + 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x69, 0x0a, 0x18, + 0x67, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, + 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x18, 0x67, + 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5f, 0x0a, 0x16, 0x61, 0x63, 0x74, 0x69, 0x76, + 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, + 0x65, 0x18, 0x1f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x63, + 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, + 0x67, 0x65, 0x52, 0x14, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x67, 0x65, 0x74, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, - 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x52, 0x1c, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, + 0x65, 0x52, 0x14, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x75, 0x0a, 0x1c, 0x67, 0x65, 0x74, 0x54, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x3c, 0x0a, 0x09, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x22, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x52, 0x09, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x54, - 0x0a, 0x11, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x18, 0x23, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x52, 0x1c, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3c, + 0x0a, 0x09, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x22, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x52, 0x09, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x54, 0x0a, 0x11, + 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x18, 0x23, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, + 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, + 0x11, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0d, 0x61, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, + 0x6e, 0x67, 0x65, 0x18, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x52, 0x11, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0d, 0x61, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, - 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x18, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x41, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x52, - 0x0d, 0x61, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x12, 0x45, - 0x0a, 0x0c, 0x61, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x25, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x75, 0x74, 0x68, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0c, 0x61, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x61, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x41, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0b, 0x61, 0x63, - 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x67, 0x65, 0x74, - 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x27, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x67, 0x65, - 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, - 0x6c, 0x0a, 0x19, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x28, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x41, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x52, 0x0d, 0x61, + 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x12, 0x45, 0x0a, 0x0c, + 0x61, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x25, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0c, 0x61, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x61, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, + 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x67, 0x65, 0x74, 0x4f, 0x72, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x27, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x52, 0x19, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, - 0x06, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x18, 0x32, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, - 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x06, 0x6e, 0x65, 0x77, 0x54, - 0x78, 0x6e, 0x12, 0x4b, 0x0a, 0x0e, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x33, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, - 0x0e, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x54, 0x0a, 0x11, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, - 0x6f, 0x54, 0x78, 0x6e, 0x18, 0x34, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, - 0x78, 0x6e, 0x52, 0x11, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x12, 0x6c, 0x0a, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x18, 0x35, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x67, 0x65, 0x74, 0x4f, + 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x6c, 0x0a, + 0x19, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x28, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x52, 0x19, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x6e, + 0x65, 0x77, 0x54, 0x78, 0x6e, 0x18, 0x32, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, + 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x06, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, + 0x12, 0x4b, 0x0a, 0x0e, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x18, 0x33, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4e, + 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0e, 0x6e, + 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, + 0x11, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, + 0x78, 0x6e, 0x18, 0x34, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x18, 0x36, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x14, 0x61, 0x64, - 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, - 0x78, 0x6e, 0x12, 0x75, 0x0a, 0x1c, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x18, 0x37, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, - 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, - 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x61, 0x64, 0x64, + 0x52, 0x11, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, + 0x54, 0x78, 0x6e, 0x12, 0x6c, 0x0a, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x18, 0x35, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x18, 0x36, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x14, 0x61, 0x64, 0x64, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, + 0x12, 0x75, 0x0a, 0x1c, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x18, 0x37, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x65, 0x6e, 0x64, - 0x54, 0x78, 0x6e, 0x18, 0x38, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x06, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x4b, - 0x0a, 0x0e, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x18, 0x39, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, - 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0e, 0x65, 0x6e, 0x64, - 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x65, - 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x3a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, - 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, - 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x6c, 0x0a, 0x19, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3b, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x61, 0x64, 0x64, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x65, 0x6e, 0x64, 0x54, 0x78, + 0x6e, 0x18, 0x38, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, + 0x64, 0x54, 0x78, 0x6e, 0x52, 0x06, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x4b, 0x0a, 0x0e, + 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x39, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, - 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x5d, 0x0a, 0x14, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x3c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, - 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x14, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, - 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x75, - 0x0a, 0x1c, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3d, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0e, 0x65, 0x6e, 0x64, 0x54, 0x78, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x65, 0x6e, 0x64, + 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x3a, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, - 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, - 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x63, 0x0a, 0x16, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, - 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, - 0x3e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x52, 0x16, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x66, 0x0a, 0x17, 0x74, 0x63, - 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x17, 0x74, 0x63, 0x43, 0x6c, 0x69, - 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x4b, 0x0a, 0x0e, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x4c, 0x69, 0x73, 0x74, 0x18, 0x40, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x52, - 0x0e, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x12, - 0x60, 0x0a, 0x15, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, - 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x41, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, - 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, - 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x15, 0x77, 0x61, 0x74, 0x63, - 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, - 0x73, 0x12, 0x51, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x18, 0x42, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x75, + 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x65, 0x6e, + 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x6c, 0x0a, 0x19, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3b, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, + 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x52, 0x19, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, + 0x14, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x3c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x52, 0x10, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x13, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x43, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x28, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x52, 0x13, 0x77, 0x61, 0x74, - 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, - 0x22, 0x86, 0x0a, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x4f, 0x4e, - 0x4e, 0x45, 0x43, 0x54, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, - 0x54, 0x45, 0x44, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, - 0x42, 0x45, 0x10, 0x04, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, - 0x10, 0x05, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x45, 0x4e, 0x44, 0x10, 0x06, 0x12, 0x10, 0x0a, 0x0c, - 0x53, 0x45, 0x4e, 0x44, 0x5f, 0x52, 0x45, 0x43, 0x45, 0x49, 0x50, 0x54, 0x10, 0x07, 0x12, 0x0e, - 0x0a, 0x0a, 0x53, 0x45, 0x4e, 0x44, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x08, 0x12, 0x0b, - 0x0a, 0x07, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x41, - 0x43, 0x4b, 0x10, 0x0a, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x0b, 0x12, 0x0f, - 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x42, 0x45, 0x10, 0x0c, 0x12, - 0x0b, 0x0a, 0x07, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, 0x53, 0x10, 0x0d, 0x12, 0x09, 0x0a, 0x05, - 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x0e, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4c, 0x4f, 0x53, 0x45, - 0x5f, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x10, 0x0f, 0x12, 0x12, 0x0a, 0x0e, 0x43, - 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x10, 0x10, 0x12, - 0x14, 0x0a, 0x10, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x43, 0x43, - 0x45, 0x53, 0x53, 0x10, 0x11, 0x12, 0x08, 0x0a, 0x04, 0x50, 0x49, 0x4e, 0x47, 0x10, 0x12, 0x12, - 0x08, 0x0a, 0x04, 0x50, 0x4f, 0x4e, 0x47, 0x10, 0x13, 0x12, 0x25, 0x0a, 0x21, 0x52, 0x45, 0x44, - 0x45, 0x4c, 0x49, 0x56, 0x45, 0x52, 0x5f, 0x55, 0x4e, 0x41, 0x43, 0x4b, 0x4e, 0x4f, 0x57, 0x4c, - 0x45, 0x44, 0x47, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x53, 0x10, 0x14, - 0x12, 0x18, 0x0a, 0x14, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, - 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, 0x10, 0x15, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x41, - 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, - 0x54, 0x41, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x16, 0x12, 0x0a, 0x0a, - 0x06, 0x4c, 0x4f, 0x4f, 0x4b, 0x55, 0x50, 0x10, 0x17, 0x12, 0x13, 0x0a, 0x0f, 0x4c, 0x4f, 0x4f, - 0x4b, 0x55, 0x50, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x18, 0x12, 0x12, - 0x0a, 0x0e, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x53, - 0x10, 0x19, 0x12, 0x1b, 0x0a, 0x17, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, - 0x54, 0x41, 0x54, 0x53, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1a, 0x12, - 0x18, 0x0a, 0x14, 0x52, 0x45, 0x41, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x45, 0x4e, 0x44, 0x5f, 0x4f, - 0x46, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x10, 0x1b, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x45, 0x45, - 0x4b, 0x10, 0x1c, 0x12, 0x17, 0x0a, 0x13, 0x47, 0x45, 0x54, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, - 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x1d, 0x12, 0x20, 0x0a, 0x1c, - 0x47, 0x45, 0x54, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, - 0x5f, 0x49, 0x44, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1e, 0x12, 0x1a, - 0x0a, 0x16, 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, - 0x52, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x10, 0x1f, 0x12, 0x1b, 0x0a, 0x17, 0x47, 0x45, - 0x54, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x53, 0x5f, 0x4f, 0x46, 0x5f, 0x4e, 0x41, 0x4d, 0x45, - 0x53, 0x50, 0x41, 0x43, 0x45, 0x10, 0x20, 0x12, 0x24, 0x0a, 0x20, 0x47, 0x45, 0x54, 0x5f, 0x54, - 0x4f, 0x50, 0x49, 0x43, 0x53, 0x5f, 0x4f, 0x46, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, - 0x43, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x21, 0x12, 0x0e, 0x0a, - 0x0a, 0x47, 0x45, 0x54, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x22, 0x12, 0x17, 0x0a, - 0x13, 0x47, 0x45, 0x54, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x52, 0x45, 0x53, 0x50, - 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x23, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x55, 0x54, 0x48, 0x5f, 0x43, - 0x48, 0x41, 0x4c, 0x4c, 0x45, 0x4e, 0x47, 0x45, 0x10, 0x24, 0x12, 0x11, 0x0a, 0x0d, 0x41, 0x55, - 0x54, 0x48, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x25, 0x12, 0x10, 0x0a, - 0x0c, 0x41, 0x43, 0x4b, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x26, 0x12, - 0x18, 0x0a, 0x14, 0x47, 0x45, 0x54, 0x5f, 0x4f, 0x52, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, - 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x27, 0x12, 0x21, 0x0a, 0x1d, 0x47, 0x45, 0x54, - 0x5f, 0x4f, 0x52, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, - 0x41, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x28, 0x12, 0x0b, 0x0a, 0x07, - 0x4e, 0x45, 0x57, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x32, 0x12, 0x14, 0x0a, 0x10, 0x4e, 0x45, 0x57, - 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x33, 0x12, - 0x18, 0x0a, 0x14, 0x41, 0x44, 0x44, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, - 0x5f, 0x54, 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x34, 0x12, 0x21, 0x0a, 0x1d, 0x41, 0x44, 0x44, - 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, 0x5f, 0x54, 0x58, - 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x35, 0x12, 0x1b, 0x0a, 0x17, - 0x41, 0x44, 0x44, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, - 0x5f, 0x54, 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x36, 0x12, 0x24, 0x0a, 0x20, 0x41, 0x44, 0x44, - 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, - 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x37, 0x12, - 0x0b, 0x0a, 0x07, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x38, 0x12, 0x14, 0x0a, 0x10, - 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, - 0x10, 0x39, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, - 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3a, 0x12, 0x21, 0x0a, 0x1d, - 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, - 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3b, 0x12, - 0x1b, 0x0a, 0x17, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x55, - 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3c, 0x12, 0x24, 0x0a, 0x20, - 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, - 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, - 0x10, 0x3d, 0x12, 0x1d, 0x0a, 0x19, 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, - 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x10, - 0x3e, 0x12, 0x1e, 0x0a, 0x1a, 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x43, - 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, - 0x3f, 0x12, 0x14, 0x0a, 0x10, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, - 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x10, 0x40, 0x12, 0x1c, 0x0a, 0x18, 0x57, 0x41, 0x54, 0x43, 0x48, - 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x5f, 0x53, 0x55, 0x43, 0x43, - 0x45, 0x53, 0x53, 0x10, 0x41, 0x12, 0x16, 0x0a, 0x12, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, - 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x42, 0x12, 0x1a, 0x0a, - 0x16, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, - 0x54, 0x5f, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x10, 0x43, 0x2a, 0x44, 0x0a, 0x0f, 0x43, 0x6f, 0x6d, - 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, - 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x4c, 0x5a, 0x34, 0x10, 0x01, 0x12, - 0x08, 0x0a, 0x04, 0x5a, 0x4c, 0x49, 0x42, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x5a, 0x53, 0x54, - 0x44, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x4e, 0x41, 0x50, 0x50, 0x59, 0x10, 0x04, 0x2a, - 0x5f, 0x0a, 0x12, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x41, 0x63, 0x63, 0x65, 0x73, - 0x73, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x10, - 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x10, 0x01, - 0x12, 0x14, 0x0a, 0x10, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x45, 0x78, 0x63, 0x6c, 0x75, - 0x73, 0x69, 0x76, 0x65, 0x10, 0x02, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, - 0x69, 0x76, 0x65, 0x57, 0x69, 0x74, 0x68, 0x46, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x10, 0x03, - 0x2a, 0x82, 0x05, 0x0a, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, - 0x12, 0x10, 0x0a, 0x0c, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, - 0x10, 0x00, 0x12, 0x11, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x72, - 0x72, 0x6f, 0x72, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, - 0x65, 0x6e, 0x63, 0x65, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x02, 0x12, 0x17, 0x0a, 0x13, 0x41, - 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, - 0x6f, 0x72, 0x10, 0x03, 0x12, 0x16, 0x0a, 0x12, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x04, 0x12, 0x10, 0x0a, 0x0c, - 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x42, 0x75, 0x73, 0x79, 0x10, 0x05, 0x12, 0x13, - 0x0a, 0x0f, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x6f, 0x74, 0x52, 0x65, 0x61, 0x64, - 0x79, 0x10, 0x06, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x42, - 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x45, 0x78, 0x63, 0x65, 0x65, - 0x64, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x07, 0x12, 0x29, 0x0a, 0x25, 0x50, 0x72, - 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x51, 0x75, 0x6f, - 0x74, 0x61, 0x45, 0x78, 0x63, 0x65, 0x65, 0x64, 0x65, 0x64, 0x45, 0x78, 0x63, 0x65, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, - 0x6d, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x09, 0x12, 0x1b, 0x0a, 0x17, 0x55, 0x6e, 0x73, 0x75, - 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x45, 0x72, - 0x72, 0x6f, 0x72, 0x10, 0x0a, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x6f, - 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0b, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, - 0x10, 0x0c, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x6f, - 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0d, 0x12, 0x13, 0x0a, 0x0f, 0x54, 0x6f, 0x6f, 0x4d, - 0x61, 0x6e, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x10, 0x0e, 0x12, 0x18, 0x0a, - 0x14, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, - 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x0f, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x65, 0x72, 0x42, 0x75, 0x73, 0x79, 0x10, 0x10, 0x12, 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, - 0x61, 0x6c, 0x69, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x11, 0x12, - 0x16, 0x0a, 0x12, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x10, 0x12, 0x12, 0x17, 0x0a, 0x13, 0x43, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x13, - 0x12, 0x22, 0x0a, 0x1e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, - 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, - 0x6e, 0x64, 0x10, 0x14, 0x12, 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x54, - 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x10, 0x15, 0x12, 0x13, 0x0a, 0x0f, 0x4e, 0x6f, - 0x74, 0x41, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x16, 0x12, - 0x17, 0x0a, 0x13, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x10, 0x17, 0x12, 0x17, 0x0a, 0x13, 0x54, 0x72, 0x61, 0x6e, - 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, - 0x18, 0x12, 0x12, 0x0a, 0x0e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x46, 0x65, 0x6e, - 0x63, 0x65, 0x64, 0x10, 0x19, 0x2a, 0x4b, 0x0a, 0x0a, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, - 0x68, 0x6f, 0x64, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, - 0x64, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x41, 0x75, 0x74, 0x68, 0x4d, - 0x65, 0x74, 0x68, 0x6f, 0x64, 0x59, 0x63, 0x61, 0x56, 0x31, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, - 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x41, 0x74, 0x68, 0x65, 0x6e, 0x73, - 0x10, 0x02, 0x2a, 0xbb, 0x01, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x56, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x30, 0x10, 0x00, 0x12, 0x06, - 0x0a, 0x02, 0x76, 0x31, 0x10, 0x01, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x32, 0x10, 0x02, 0x12, 0x06, - 0x0a, 0x02, 0x76, 0x33, 0x10, 0x03, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x34, 0x10, 0x04, 0x12, 0x06, - 0x0a, 0x02, 0x76, 0x35, 0x10, 0x05, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x36, 0x10, 0x06, 0x12, 0x06, - 0x0a, 0x02, 0x76, 0x37, 0x10, 0x07, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x38, 0x10, 0x08, 0x12, 0x06, - 0x0a, 0x02, 0x76, 0x39, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x30, 0x10, 0x0a, 0x12, - 0x07, 0x0a, 0x03, 0x76, 0x31, 0x31, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x32, 0x10, - 0x0c, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x33, 0x10, 0x0d, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, - 0x34, 0x10, 0x0e, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x35, 0x10, 0x0f, 0x12, 0x07, 0x0a, 0x03, - 0x76, 0x31, 0x36, 0x10, 0x10, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x37, 0x10, 0x11, 0x12, 0x07, - 0x0a, 0x03, 0x76, 0x31, 0x38, 0x10, 0x12, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x39, 0x10, 0x13, - 0x2a, 0x2b, 0x0a, 0x0d, 0x4b, 0x65, 0x79, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x4d, 0x6f, 0x64, - 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x41, 0x55, 0x54, 0x4f, 0x5f, 0x53, 0x50, 0x4c, 0x49, 0x54, 0x10, - 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x54, 0x49, 0x43, 0x4b, 0x59, 0x10, 0x01, 0x2a, 0x22, 0x0a, - 0x09, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x4f, - 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x10, - 0x01, 0x42, 0x2f, 0x0a, 0x22, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, - 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x61, 0x70, - 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x48, 0x03, 0x5a, 0x07, 0x2e, 0x2f, 0x70, 0x72, 0x6f, - 0x74, 0x6f, + 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x14, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, + 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x75, 0x0a, 0x1c, + 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3d, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, + 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x63, 0x0a, 0x16, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x3e, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x52, 0x16, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x66, 0x0a, 0x17, 0x74, 0x63, 0x43, 0x6c, + 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x18, 0x3f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x17, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, + 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x4b, 0x0a, 0x0e, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, + 0x73, 0x74, 0x18, 0x40, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, + 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x0e, 0x77, + 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x60, 0x0a, + 0x15, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, + 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x41, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x70, + 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, + 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x15, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, + 0x51, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x18, 0x42, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x52, 0x10, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x13, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x43, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x28, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x52, 0x13, 0x77, 0x61, 0x74, 0x63, 0x68, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0x86, + 0x0a, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x4f, 0x4e, 0x4e, 0x45, + 0x43, 0x54, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x45, + 0x44, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x42, 0x45, + 0x10, 0x04, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x10, 0x05, + 0x12, 0x08, 0x0a, 0x04, 0x53, 0x45, 0x4e, 0x44, 0x10, 0x06, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x45, + 0x4e, 0x44, 0x5f, 0x52, 0x45, 0x43, 0x45, 0x49, 0x50, 0x54, 0x10, 0x07, 0x12, 0x0e, 0x0a, 0x0a, + 0x53, 0x45, 0x4e, 0x44, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x08, 0x12, 0x0b, 0x0a, 0x07, + 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x43, 0x4b, + 0x10, 0x0a, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x0b, 0x12, 0x0f, 0x0a, 0x0b, + 0x55, 0x4e, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x42, 0x45, 0x10, 0x0c, 0x12, 0x0b, 0x0a, + 0x07, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, 0x53, 0x10, 0x0d, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x52, + 0x52, 0x4f, 0x52, 0x10, 0x0e, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x50, + 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x10, 0x0f, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4c, 0x4f, + 0x53, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x10, 0x10, 0x12, 0x14, 0x0a, + 0x10, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, + 0x53, 0x10, 0x11, 0x12, 0x08, 0x0a, 0x04, 0x50, 0x49, 0x4e, 0x47, 0x10, 0x12, 0x12, 0x08, 0x0a, + 0x04, 0x50, 0x4f, 0x4e, 0x47, 0x10, 0x13, 0x12, 0x25, 0x0a, 0x21, 0x52, 0x45, 0x44, 0x45, 0x4c, + 0x49, 0x56, 0x45, 0x52, 0x5f, 0x55, 0x4e, 0x41, 0x43, 0x4b, 0x4e, 0x4f, 0x57, 0x4c, 0x45, 0x44, + 0x47, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x53, 0x10, 0x14, 0x12, 0x18, + 0x0a, 0x14, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x4d, 0x45, + 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, 0x10, 0x15, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x41, 0x52, 0x54, + 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, + 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x16, 0x12, 0x0a, 0x0a, 0x06, 0x4c, + 0x4f, 0x4f, 0x4b, 0x55, 0x50, 0x10, 0x17, 0x12, 0x13, 0x0a, 0x0f, 0x4c, 0x4f, 0x4f, 0x4b, 0x55, + 0x50, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x18, 0x12, 0x12, 0x0a, 0x0e, + 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x53, 0x10, 0x19, + 0x12, 0x1b, 0x0a, 0x17, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, + 0x54, 0x53, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1a, 0x12, 0x18, 0x0a, + 0x14, 0x52, 0x45, 0x41, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x45, 0x4e, 0x44, 0x5f, 0x4f, 0x46, 0x5f, + 0x54, 0x4f, 0x50, 0x49, 0x43, 0x10, 0x1b, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x45, 0x45, 0x4b, 0x10, + 0x1c, 0x12, 0x17, 0x0a, 0x13, 0x47, 0x45, 0x54, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, 0x4d, 0x45, + 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x1d, 0x12, 0x20, 0x0a, 0x1c, 0x47, 0x45, + 0x54, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x49, + 0x44, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1e, 0x12, 0x1a, 0x0a, 0x16, + 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, + 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x10, 0x1f, 0x12, 0x1b, 0x0a, 0x17, 0x47, 0x45, 0x54, 0x5f, + 0x54, 0x4f, 0x50, 0x49, 0x43, 0x53, 0x5f, 0x4f, 0x46, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, + 0x41, 0x43, 0x45, 0x10, 0x20, 0x12, 0x24, 0x0a, 0x20, 0x47, 0x45, 0x54, 0x5f, 0x54, 0x4f, 0x50, + 0x49, 0x43, 0x53, 0x5f, 0x4f, 0x46, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, + 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x21, 0x12, 0x0e, 0x0a, 0x0a, 0x47, + 0x45, 0x54, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x22, 0x12, 0x17, 0x0a, 0x13, 0x47, + 0x45, 0x54, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, + 0x53, 0x45, 0x10, 0x23, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x55, 0x54, 0x48, 0x5f, 0x43, 0x48, 0x41, + 0x4c, 0x4c, 0x45, 0x4e, 0x47, 0x45, 0x10, 0x24, 0x12, 0x11, 0x0a, 0x0d, 0x41, 0x55, 0x54, 0x48, + 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x25, 0x12, 0x10, 0x0a, 0x0c, 0x41, + 0x43, 0x4b, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x26, 0x12, 0x18, 0x0a, + 0x14, 0x47, 0x45, 0x54, 0x5f, 0x4f, 0x52, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f, 0x53, + 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x27, 0x12, 0x21, 0x0a, 0x1d, 0x47, 0x45, 0x54, 0x5f, 0x4f, + 0x52, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, + 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x28, 0x12, 0x0b, 0x0a, 0x07, 0x4e, 0x45, + 0x57, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x32, 0x12, 0x14, 0x0a, 0x10, 0x4e, 0x45, 0x57, 0x5f, 0x54, + 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x33, 0x12, 0x18, 0x0a, + 0x14, 0x41, 0x44, 0x44, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, + 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x34, 0x12, 0x21, 0x0a, 0x1d, 0x41, 0x44, 0x44, 0x5f, 0x50, + 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x5f, + 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x35, 0x12, 0x1b, 0x0a, 0x17, 0x41, 0x44, + 0x44, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, + 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x36, 0x12, 0x24, 0x0a, 0x20, 0x41, 0x44, 0x44, 0x5f, 0x53, + 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, 0x5f, 0x54, + 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x37, 0x12, 0x0b, 0x0a, + 0x07, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x38, 0x12, 0x14, 0x0a, 0x10, 0x45, 0x4e, + 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x39, + 0x12, 0x18, 0x0a, 0x14, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x50, + 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3a, 0x12, 0x21, 0x0a, 0x1d, 0x45, 0x4e, + 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3b, 0x12, 0x1b, 0x0a, + 0x17, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x55, 0x42, 0x53, + 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3c, 0x12, 0x24, 0x0a, 0x20, 0x45, 0x4e, + 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, + 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3d, + 0x12, 0x1d, 0x0a, 0x19, 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4f, + 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x10, 0x3e, 0x12, + 0x1e, 0x0a, 0x1a, 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4f, 0x4e, + 0x4e, 0x45, 0x43, 0x54, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3f, 0x12, + 0x14, 0x0a, 0x10, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, + 0x49, 0x53, 0x54, 0x10, 0x40, 0x12, 0x1c, 0x0a, 0x18, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, + 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x5f, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, + 0x53, 0x10, 0x41, 0x12, 0x16, 0x0a, 0x12, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, + 0x49, 0x43, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x42, 0x12, 0x1a, 0x0a, 0x16, 0x57, + 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x5f, + 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x10, 0x43, 0x2a, 0x44, 0x0a, 0x0f, 0x43, 0x6f, 0x6d, 0x70, 0x72, + 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x4f, + 0x4e, 0x45, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x4c, 0x5a, 0x34, 0x10, 0x01, 0x12, 0x08, 0x0a, + 0x04, 0x5a, 0x4c, 0x49, 0x42, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x5a, 0x53, 0x54, 0x44, 0x10, + 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x4e, 0x41, 0x50, 0x50, 0x59, 0x10, 0x04, 0x2a, 0x5f, 0x0a, + 0x12, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4d, + 0x6f, 0x64, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x10, 0x00, 0x12, + 0x0d, 0x0a, 0x09, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x10, 0x01, 0x12, 0x14, + 0x0a, 0x10, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, + 0x76, 0x65, 0x10, 0x02, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, + 0x65, 0x57, 0x69, 0x74, 0x68, 0x46, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x2a, 0x82, + 0x05, 0x0a, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x10, + 0x0a, 0x0c, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x00, + 0x12, 0x11, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x72, 0x72, 0x6f, + 0x72, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, + 0x63, 0x65, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x02, 0x12, 0x17, 0x0a, 0x13, 0x41, 0x75, 0x74, + 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x10, 0x03, 0x12, 0x16, 0x0a, 0x12, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x04, 0x12, 0x10, 0x0a, 0x0c, 0x43, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x42, 0x75, 0x73, 0x79, 0x10, 0x05, 0x12, 0x13, 0x0a, 0x0f, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x6f, 0x74, 0x52, 0x65, 0x61, 0x64, 0x79, 0x10, + 0x06, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x42, 0x6c, 0x6f, + 0x63, 0x6b, 0x65, 0x64, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x45, 0x78, 0x63, 0x65, 0x65, 0x64, 0x65, + 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x07, 0x12, 0x29, 0x0a, 0x25, 0x50, 0x72, 0x6f, 0x64, + 0x75, 0x63, 0x65, 0x72, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x51, 0x75, 0x6f, 0x74, 0x61, + 0x45, 0x78, 0x63, 0x65, 0x65, 0x64, 0x65, 0x64, 0x45, 0x78, 0x63, 0x65, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x10, 0x08, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x45, + 0x72, 0x72, 0x6f, 0x72, 0x10, 0x09, 0x12, 0x1b, 0x0a, 0x17, 0x55, 0x6e, 0x73, 0x75, 0x70, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, + 0x72, 0x10, 0x0a, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x6f, 0x74, 0x46, + 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0b, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0c, + 0x12, 0x14, 0x0a, 0x10, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x6f, 0x74, 0x46, + 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0d, 0x12, 0x13, 0x0a, 0x0f, 0x54, 0x6f, 0x6f, 0x4d, 0x61, 0x6e, + 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x10, 0x0e, 0x12, 0x18, 0x0a, 0x14, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x45, 0x72, + 0x72, 0x6f, 0x72, 0x10, 0x0f, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, + 0x72, 0x42, 0x75, 0x73, 0x79, 0x10, 0x10, 0x12, 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, 0x61, 0x6c, + 0x69, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x11, 0x12, 0x16, 0x0a, + 0x12, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x10, 0x12, 0x12, 0x17, 0x0a, 0x13, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x13, 0x12, 0x22, + 0x0a, 0x1e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6f, + 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, + 0x10, 0x14, 0x12, 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x54, 0x78, 0x6e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x10, 0x15, 0x12, 0x13, 0x0a, 0x0f, 0x4e, 0x6f, 0x74, 0x41, + 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x16, 0x12, 0x17, 0x0a, + 0x13, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, + 0x6c, 0x69, 0x63, 0x74, 0x10, 0x17, 0x12, 0x17, 0x0a, 0x13, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x18, 0x12, + 0x12, 0x0a, 0x0e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x46, 0x65, 0x6e, 0x63, 0x65, + 0x64, 0x10, 0x19, 0x2a, 0x4b, 0x0a, 0x0a, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, + 0x64, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4e, + 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, + 0x68, 0x6f, 0x64, 0x59, 0x63, 0x61, 0x56, 0x31, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x41, 0x75, + 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x41, 0x74, 0x68, 0x65, 0x6e, 0x73, 0x10, 0x02, + 0x2a, 0xbb, 0x01, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x30, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, + 0x76, 0x31, 0x10, 0x01, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x32, 0x10, 0x02, 0x12, 0x06, 0x0a, 0x02, + 0x76, 0x33, 0x10, 0x03, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x34, 0x10, 0x04, 0x12, 0x06, 0x0a, 0x02, + 0x76, 0x35, 0x10, 0x05, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x36, 0x10, 0x06, 0x12, 0x06, 0x0a, 0x02, + 0x76, 0x37, 0x10, 0x07, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x38, 0x10, 0x08, 0x12, 0x06, 0x0a, 0x02, + 0x76, 0x39, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x30, 0x10, 0x0a, 0x12, 0x07, 0x0a, + 0x03, 0x76, 0x31, 0x31, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x32, 0x10, 0x0c, 0x12, + 0x07, 0x0a, 0x03, 0x76, 0x31, 0x33, 0x10, 0x0d, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x34, 0x10, + 0x0e, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x35, 0x10, 0x0f, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, + 0x36, 0x10, 0x10, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x37, 0x10, 0x11, 0x12, 0x07, 0x0a, 0x03, + 0x76, 0x31, 0x38, 0x10, 0x12, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x39, 0x10, 0x13, 0x2a, 0x2b, + 0x0a, 0x0d, 0x4b, 0x65, 0x79, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x4d, 0x6f, 0x64, 0x65, 0x12, + 0x0e, 0x0a, 0x0a, 0x41, 0x55, 0x54, 0x4f, 0x5f, 0x53, 0x50, 0x4c, 0x49, 0x54, 0x10, 0x00, 0x12, + 0x0a, 0x0a, 0x06, 0x53, 0x54, 0x49, 0x43, 0x4b, 0x59, 0x10, 0x01, 0x2a, 0x22, 0x0a, 0x09, 0x54, + 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x4f, 0x4d, 0x4d, + 0x49, 0x54, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x10, 0x01, 0x42, + 0x2f, 0x0a, 0x22, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x70, 0x75, + 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x48, 0x03, 0x5a, 0x07, 0x2e, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, } var ( diff --git a/pulsar/internal/pulsar_proto/PulsarApi.proto b/pulsar/internal/pulsar_proto/PulsarApi.proto index 8bd75729c8..a95a114802 100644 --- a/pulsar/internal/pulsar_proto/PulsarApi.proto +++ b/pulsar/internal/pulsar_proto/PulsarApi.proto @@ -625,11 +625,15 @@ message CommandReachedEndOfTopic { message CommandCloseProducer { required uint64 producer_id = 1; required uint64 request_id = 2; + optional string assignedBrokerServiceUrl = 3; + optional string assignedBrokerServiceUrlTls = 4; } message CommandCloseConsumer { required uint64 consumer_id = 1; required uint64 request_id = 2; + optional string assignedBrokerServiceUrl = 3; + optional string assignedBrokerServiceUrlTls = 4; } message CommandRedeliverUnacknowledgedMessages { diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index fbcc5b9776..f5c838c4d3 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -107,7 +107,7 @@ type partitionProducer struct { // Channel where app is posting messages to be published dataChan chan *sendRequest cmdChan chan interface{} - connectClosedCh chan connectionClosed + connectClosedCh chan *connectionClosed publishSemaphore internal.Semaphore pendingQueue internal.BlockingQueue @@ -168,7 +168,7 @@ func newPartitionProducer(client *client, topic string, options *ProducerOptions producerID: client.rpcClient.NewProducerID(), dataChan: make(chan *sendRequest, maxPendingMessages), cmdChan: make(chan interface{}, 10), - connectClosedCh: make(chan connectionClosed, 10), + connectClosedCh: make(chan *connectionClosed, 10), batchFlushTicker: time.NewTicker(batchingMaxPublishDelay), compressionProvider: internal.GetCompressionProvider(pb.CompressionType(options.CompressionType), compression.Level(options.CompressionLevel)), @@ -197,7 +197,7 @@ func newPartitionProducer(client *client, topic string, options *ProducerOptions } else { p.userProvidedProducerName = false } - err := p.grabCnx() + err := p.grabCnx("") if err != nil { p.batchFlushTicker.Stop() logger.WithError(err).Error("Failed to create producer at newPartitionProducer") @@ -221,14 +221,25 @@ func newPartitionProducer(client *client, topic string, options *ProducerOptions return p, nil } -func (p *partitionProducer) grabCnx() error { - lr, err := p.client.lookupService.Lookup(p.topic) +func (p *partitionProducer) lookupTopic(brokerServiceURL string) (*internal.LookupResult, error) { + if len(brokerServiceURL) == 0 { + lr, err := p.client.lookupService.Lookup(p.topic) + if err != nil { + p.log.WithError(err).Warn("Failed to lookup topic") + return nil, err + } + + p.log.Debug("Lookup result: ", lr) + return lr, err + } + return p.client.lookupService.GetBrokerAddress(brokerServiceURL, p._getConn().IsProxied()) +} + +func (p *partitionProducer) grabCnx(assignedBrokerURL string) error { + lr, err := p.lookupTopic(assignedBrokerURL) if err != nil { - p.log.WithError(err).Warn("Failed to lookup topic") return err } - - p.log.Debug("Lookup result: ", lr) id := p.client.rpcClient.NewRequestID() // set schema info for producer @@ -363,7 +374,13 @@ func (p *partitionProducer) grabCnx() error { return nil } -type connectionClosed struct{} +type connectionClosed struct { + assignedBrokerURL string +} + +func (cc *connectionClosed) HasURL() bool { + return len(cc.assignedBrokerURL) > 0 +} func (p *partitionProducer) GetBuffer() internal.Buffer { b, ok := buffersPool.Get().(internal.Buffer) @@ -373,10 +390,17 @@ func (p *partitionProducer) GetBuffer() internal.Buffer { return b } -func (p *partitionProducer) ConnectionClosed() { +func (p *partitionProducer) ConnectionClosed(closeProducer *pb.CommandCloseProducer) { // Trigger reconnection in the produce goroutine p.log.WithField("cnx", p._getConn().ID()).Warn("Connection was closed") - p.connectClosedCh <- connectionClosed{} + var assignedBrokerURL string + if closeProducer != nil { + assignedBrokerURL = p.client.selectServiceURL( + closeProducer.GetAssignedBrokerServiceUrl(), closeProducer.GetAssignedBrokerServiceUrlTls()) + } + p.connectClosedCh <- &connectionClosed{ + assignedBrokerURL: assignedBrokerURL, + } } func (p *partitionProducer) getOrCreateSchema(schemaInfo *SchemaInfo) (schemaVersion []byte, err error) { @@ -409,7 +433,7 @@ func (p *partitionProducer) getOrCreateSchema(schemaInfo *SchemaInfo) (schemaVer return res.Response.GetOrCreateSchemaResponse.SchemaVersion, nil } -func (p *partitionProducer) reconnectToBroker() { +func (p *partitionProducer) reconnectToBroker(connectionClosed *connectionClosed) { var maxRetry int if p.options.MaxReconnectToBroker == nil { maxRetry = -1 @@ -429,12 +453,22 @@ func (p *partitionProducer) reconnectToBroker() { return } - if p.options.BackoffPolicy == nil { + var assignedBrokerURL string + + if connectionClosed != nil && connectionClosed.HasURL() { + delayReconnectTime = 0 + assignedBrokerURL = connectionClosed.assignedBrokerURL + connectionClosed = nil // Only attempt once + } else if p.options.BackoffPolicy == nil { delayReconnectTime = defaultBackoff.Next() } else { delayReconnectTime = p.options.BackoffPolicy.Next() } - p.log.Info("Reconnecting to broker in ", delayReconnectTime) + + p.log.WithFields(log.Fields{ + "assignedBrokerURL": assignedBrokerURL, + "delayReconnectTime": delayReconnectTime, + }).Info("Reconnecting to broker") time.Sleep(delayReconnectTime) // double check @@ -445,7 +479,7 @@ func (p *partitionProducer) reconnectToBroker() { } atomic.AddUint64(&p.epoch, 1) - err := p.grabCnx() + err := p.grabCnx(assignedBrokerURL) if err == nil { // Successfully reconnected p.log.WithField("cnx", p._getConn().ID()).Info("Reconnected producer to broker") @@ -509,9 +543,9 @@ func (p *partitionProducer) runEventsLoop() { p.internalClose(v) return } - case <-p.connectClosedCh: + case connectionClosed := <-p.connectClosedCh: p.log.Info("runEventsLoop will reconnect in producer") - p.reconnectToBroker() + p.reconnectToBroker(connectionClosed) case <-p.batchFlushTicker.C: p.internalFlushCurrentBatch() } diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index ba5911565e..7c4ff89752 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -1285,22 +1285,22 @@ func TestProducerWithBackoffPolicy(t *testing.T) { partitionProducerImp := _producer.(*producer).producers[0].(*partitionProducer) // 1 s startTime := time.Now() - partitionProducerImp.reconnectToBroker() + partitionProducerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 2 s startTime = time.Now() - partitionProducerImp.reconnectToBroker() + partitionProducerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionProducerImp.reconnectToBroker() + partitionProducerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionProducerImp.reconnectToBroker() + partitionProducerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) } diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index 93787d106c..a9c45ba806 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -878,22 +878,22 @@ func TestReaderWithBackoffPolicy(t *testing.T) { partitionConsumerImp := _reader.(*reader).c.consumers[0] // 1 s startTime := time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 2 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) } diff --git a/scripts/run-ci-extensible-load-manager.sh b/scripts/run-ci-extensible-load-manager.sh new file mode 100755 index 0000000000..2fe5f7c229 --- /dev/null +++ b/scripts/run-ci-extensible-load-manager.sh @@ -0,0 +1,23 @@ +#!/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. + +set -e -x + +go test -race -coverprofile=/tmp/coverage -timeout=5m -tags extensible_load_manager -v -run TestExtensibleLoadManagerTestSuite ./pulsar +go tool cover -html=/tmp/coverage -o coverage.html + From 8fa0878c88ee4b6e002c7764a10c79012b14be90 Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Tue, 7 May 2024 11:13:11 +0800 Subject: [PATCH 341/348] [Improve] Add admin api GetListActiveBrokers (#1212) ### Motivation To keep consistent with the [Java client](/~https://github.com/apache/pulsar/pull/14702). ### Modifications Add admin api GetListActiveBrokers --- pulsaradmin/pkg/admin/brokers.go | 13 +++++++++++++ pulsaradmin/pkg/admin/brokers_test.go | 15 +++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/pulsaradmin/pkg/admin/brokers.go b/pulsaradmin/pkg/admin/brokers.go index 650fab8e0b..7dcea800e3 100644 --- a/pulsaradmin/pkg/admin/brokers.go +++ b/pulsaradmin/pkg/admin/brokers.go @@ -27,6 +27,9 @@ import ( // Brokers is admin interface for brokers management type Brokers interface { + + // GetListActiveBrokers Get the list of active brokers in the local cluster. + GetListActiveBrokers() ([]string, error) // GetActiveBrokers returns the list of active brokers in the cluster. GetActiveBrokers(cluster string) ([]string, error) @@ -86,6 +89,16 @@ func (b *broker) GetActiveBrokers(cluster string) ([]string, error) { return res, nil } +func (b *broker) GetListActiveBrokers() ([]string, error) { + endpoint := b.pulsar.endpoint(b.basePath) + var res []string + err := b.pulsar.Client.Get(endpoint, &res) + if err != nil { + return nil, err + } + return res, nil +} + func (b *broker) GetDynamicConfigurationNames() ([]string, error) { endpoint := b.pulsar.endpoint(b.basePath, "/configuration/") var res []string diff --git a/pulsaradmin/pkg/admin/brokers_test.go b/pulsaradmin/pkg/admin/brokers_test.go index 9767975905..3ae9e4aec9 100644 --- a/pulsaradmin/pkg/admin/brokers_test.go +++ b/pulsaradmin/pkg/admin/brokers_test.go @@ -58,3 +58,18 @@ func TestGetLeaderBroker(t *testing.T) { assert.NotEmpty(t, leaderBroker.ServiceURL) assert.NotEmpty(t, leaderBroker.BrokerID) } + +func TestGetAllActiveBrokers(t *testing.T) { + readFile, err := os.ReadFile("../../../integration-tests/tokens/admin-token") + assert.NoError(t, err) + cfg := &config.Config{ + Token: string(readFile), + } + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + + brokers, err := admin.Brokers().GetListActiveBrokers() + assert.NoError(t, err) + assert.NotEmpty(t, brokers) +} From a086d3257cfbe1226ca9727211de449857b178e4 Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Tue, 7 May 2024 23:13:47 +0800 Subject: [PATCH 342/348] [Improve] Add admin api ForceDeleteSchema (#1213) ### Motivation To keep consistent with the Java client. ### Modifications Add admin api ForceDeleteSchema --- pulsaradmin/pkg/admin/schema.go | 16 +++++- pulsaradmin/pkg/admin/schema_test.go | 79 ++++++++++++++++++++++++++++ 2 files changed, 93 insertions(+), 2 deletions(-) create mode 100644 pulsaradmin/pkg/admin/schema_test.go diff --git a/pulsaradmin/pkg/admin/schema.go b/pulsaradmin/pkg/admin/schema.go index 1465526841..d97cd7adc5 100644 --- a/pulsaradmin/pkg/admin/schema.go +++ b/pulsaradmin/pkg/admin/schema.go @@ -38,6 +38,9 @@ type Schema interface { // DeleteSchema deletes the schema associated with a given topic DeleteSchema(topic string) error + // ForceDeleteSchema force deletes the schema associated with a given topic + ForceDeleteSchema(topic string) error + // CreateSchemaByPayload creates a schema for a given topic CreateSchemaByPayload(topic string, schemaPayload utils.PostSchemaPayload) error } @@ -112,6 +115,14 @@ func (s *schemas) GetSchemaInfoByVersion(topic string, version int64) (*utils.Sc } func (s *schemas) DeleteSchema(topic string) error { + return s.delete(topic, false) +} + +func (s *schemas) ForceDeleteSchema(topic string) error { + return s.delete(topic, true) +} + +func (s *schemas) delete(topic string, force bool) error { topicName, err := utils.GetTopicName(topic) if err != nil { return err @@ -120,9 +131,10 @@ func (s *schemas) DeleteSchema(topic string) error { endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), topicName.GetLocalName(), "schema") - fmt.Println(endpoint) + queryParams := make(map[string]string) + queryParams["force"] = strconv.FormatBool(force) - return s.pulsar.Client.Delete(endpoint) + return s.pulsar.Client.DeleteWithQueryParams(endpoint, queryParams) } func (s *schemas) CreateSchemaByPayload(topic string, schemaPayload utils.PostSchemaPayload) error { diff --git a/pulsaradmin/pkg/admin/schema_test.go b/pulsaradmin/pkg/admin/schema_test.go new file mode 100644 index 0000000000..17c1a54dd1 --- /dev/null +++ b/pulsaradmin/pkg/admin/schema_test.go @@ -0,0 +1,79 @@ +// 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 admin + +import ( + "fmt" + "testing" + "time" + + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" + "github.com/stretchr/testify/assert" +) + +func TestSchemas_DeleteSchema(t *testing.T) { + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + + schemaPayload := utils.PostSchemaPayload{ + SchemaType: "STRING", + Schema: "", + } + topic := fmt.Sprintf("my-topic-%v", time.Now().Nanosecond()) + err = admin.Schemas().CreateSchemaByPayload(topic, schemaPayload) + assert.NoError(t, err) + + info, err := admin.Schemas().GetSchemaInfo(topic) + assert.NoError(t, err) + assert.Equal(t, schemaPayload.SchemaType, info.Type) + + err = admin.Schemas().DeleteSchema(topic) + assert.NoError(t, err) + + _, err = admin.Schemas().GetSchemaInfo(topic) + assert.Errorf(t, err, "Schema not found") + +} +func TestSchemas_ForceDeleteSchema(t *testing.T) { + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + + schemaPayload := utils.PostSchemaPayload{ + SchemaType: "STRING", + Schema: "", + } + topic := fmt.Sprintf("my-topic-%v", time.Now().Nanosecond()) + err = admin.Schemas().CreateSchemaByPayload(topic, schemaPayload) + assert.NoError(t, err) + + info, err := admin.Schemas().GetSchemaInfo(topic) + assert.NoError(t, err) + assert.Equal(t, schemaPayload.SchemaType, info.Type) + + err = admin.Schemas().ForceDeleteSchema(topic) + assert.NoError(t, err) + + _, err = admin.Schemas().GetSchemaInfo(topic) + assert.Errorf(t, err, "Schema not found") + +} From 007d14e50c886a0bbfc73d2a43d3cc932c9221d8 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Thu, 9 May 2024 13:46:42 +0800 Subject: [PATCH 343/348] [improve] Upgrade golang-jwt to v5 (#1214) --- go.mod | 2 +- go.sum | 3 ++- oauth2/auth.go | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 5225551207..b0f822f375 100644 --- a/go.mod +++ b/go.mod @@ -9,7 +9,6 @@ require ( github.com/bits-and-blooms/bitset v1.4.0 github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b github.com/davecgh/go-spew v1.1.1 - github.com/golang-jwt/jwt v3.2.1+incompatible github.com/google/uuid v1.1.2 github.com/klauspost/compress v1.14.4 github.com/linkedin/goavro/v2 v2.9.8 @@ -31,6 +30,7 @@ require ( ) require ( + github.com/golang-jwt/jwt/v5 v5.2.1 github.com/golang/protobuf v1.5.2 github.com/hashicorp/go-multierror v1.1.1 ) diff --git a/go.sum b/go.sum index f48b384ace..64f5faab99 100644 --- a/go.sum +++ b/go.sum @@ -98,8 +98,9 @@ github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg78 github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 h1:ZpnhV/YsD2/4cESfV5+Hoeu/iUR3ruzNvZ+yQfO03a0= github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2/go.mod h1:bBOAhwG1umN6/6ZUMtDFBMQR8jRg9O75tm9K00oMsK4= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/golang-jwt/jwt v3.2.1+incompatible h1:73Z+4BJcrTC+KczS6WvTPvRGOp1WmfEP4Q1lOd9Z/+c= github.com/golang-jwt/jwt v3.2.1+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= +github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17wHk= +github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= diff --git a/oauth2/auth.go b/oauth2/auth.go index d44bd35053..9f4293b5b2 100644 --- a/oauth2/auth.go +++ b/oauth2/auth.go @@ -22,7 +22,7 @@ import ( "time" "github.com/apache/pulsar-client-go/oauth2/clock" - "github.com/golang-jwt/jwt" + "github.com/golang-jwt/jwt/v5" "golang.org/x/oauth2" ) From 49fce72614c37fd8644c6305eb3405fcf48d4f52 Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Fri, 10 May 2024 22:08:57 +0800 Subject: [PATCH 344/348] [Improve] Supplement schema admin api (#1215) ### Motivation To keep consistent with the Java client. ### Modifications - CreateSchemaBySchemaInfo - GetVersionBySchemaInfo - GetVersionByPayload - TestCompatibilityWithSchemaInfo - TestCompatibilityWithPostSchemaPayload --- pulsaradmin/pkg/admin/schema.go | 59 ++++++++++++++++++++++++++++ pulsaradmin/pkg/admin/schema_test.go | 45 +++++++++++++++++++++ pulsaradmin/pkg/utils/schema_util.go | 23 +++++++++++ 3 files changed, 127 insertions(+) diff --git a/pulsaradmin/pkg/admin/schema.go b/pulsaradmin/pkg/admin/schema.go index d97cd7adc5..7190bd9961 100644 --- a/pulsaradmin/pkg/admin/schema.go +++ b/pulsaradmin/pkg/admin/schema.go @@ -43,6 +43,22 @@ type Schema interface { // CreateSchemaByPayload creates a schema for a given topic CreateSchemaByPayload(topic string, schemaPayload utils.PostSchemaPayload) error + + // CreateSchemaBySchemaInfo creates a schema for a given topic + CreateSchemaBySchemaInfo(topic string, schemaInfo utils.SchemaInfo) error + + // GetVersionBySchemaInfo gets the version of a schema + GetVersionBySchemaInfo(topic string, schemaInfo utils.SchemaInfo) (int64, error) + + // GetVersionByPayload gets the version of a schema + GetVersionByPayload(topic string, schemaPayload utils.PostSchemaPayload) (int64, error) + + // TestCompatibilityWithSchemaInfo tests compatibility with a schema + TestCompatibilityWithSchemaInfo(topic string, schemaInfo utils.SchemaInfo) (*utils.IsCompatibility, error) + + // TestCompatibilityWithPostSchemaPayload tests compatibility with a schema + TestCompatibilityWithPostSchemaPayload(topic string, + schemaPayload utils.PostSchemaPayload) (*utils.IsCompatibility, error) } type schemas struct { @@ -148,3 +164,46 @@ func (s *schemas) CreateSchemaByPayload(topic string, schemaPayload utils.PostSc return s.pulsar.Client.Post(endpoint, &schemaPayload) } + +func (s *schemas) CreateSchemaBySchemaInfo(topic string, schemaInfo utils.SchemaInfo) error { + schemaPayload := utils.ConvertSchemaInfoToPostSchemaPayload(schemaInfo) + return s.CreateSchemaByPayload(topic, schemaPayload) +} + +func (s *schemas) GetVersionBySchemaInfo(topic string, schemaInfo utils.SchemaInfo) (int64, error) { + schemaPayload := utils.ConvertSchemaInfoToPostSchemaPayload(schemaInfo) + return s.GetVersionByPayload(topic, schemaPayload) +} + +func (s *schemas) GetVersionByPayload(topic string, schemaPayload utils.PostSchemaPayload) (int64, error) { + topicName, err := utils.GetTopicName(topic) + if err != nil { + return 0, err + } + version := struct { + Version int64 `json:"version"` + }{} + endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), + topicName.GetLocalName(), "version") + err = s.pulsar.Client.PostWithObj(endpoint, &schemaPayload, &version) + return version.Version, err +} + +func (s *schemas) TestCompatibilityWithSchemaInfo(topic string, + schemaInfo utils.SchemaInfo) (*utils.IsCompatibility, error) { + schemaPayload := utils.ConvertSchemaInfoToPostSchemaPayload(schemaInfo) + return s.TestCompatibilityWithPostSchemaPayload(topic, schemaPayload) +} + +func (s *schemas) TestCompatibilityWithPostSchemaPayload(topic string, + schemaPayload utils.PostSchemaPayload) (*utils.IsCompatibility, error) { + topicName, err := utils.GetTopicName(topic) + if err != nil { + return nil, err + } + var isCompatibility utils.IsCompatibility + endpoint := s.pulsar.endpoint(s.basePath, topicName.GetTenant(), topicName.GetNamespace(), + topicName.GetLocalName(), "compatibility") + err = s.pulsar.Client.PostWithObj(endpoint, &schemaPayload, &isCompatibility) + return &isCompatibility, err +} diff --git a/pulsaradmin/pkg/admin/schema_test.go b/pulsaradmin/pkg/admin/schema_test.go index 17c1a54dd1..3560559e60 100644 --- a/pulsaradmin/pkg/admin/schema_test.go +++ b/pulsaradmin/pkg/admin/schema_test.go @@ -77,3 +77,48 @@ func TestSchemas_ForceDeleteSchema(t *testing.T) { assert.Errorf(t, err, "Schema not found") } + +func TestSchemas_CreateSchemaBySchemaInfo(t *testing.T) { + cfg := &config.Config{} + admin, err := New(cfg) + assert.NoError(t, err) + assert.NotNil(t, admin) + + schemaInfo := utils.SchemaInfo{ + Schema: []byte(""), + Type: "STRING", + } + topic := fmt.Sprintf("my-topic-%v", time.Now().Nanosecond()) + err = admin.Schemas().CreateSchemaBySchemaInfo(topic, schemaInfo) + assert.NoError(t, err) + + info, err := admin.Schemas().GetSchemaInfo(topic) + assert.NoError(t, err) + assert.Equal(t, schemaInfo.Type, info.Type) + + version, err := admin.Schemas().GetVersionBySchemaInfo(topic, schemaInfo) + assert.NoError(t, err) + assert.Equal(t, version, int64(0)) + + schemaPayload := utils.ConvertSchemaInfoToPostSchemaPayload(schemaInfo) + version, err = admin.Schemas().GetVersionByPayload(topic, schemaPayload) + assert.NoError(t, err) + assert.Equal(t, version, int64(0)) + + compatibility, err := admin.Schemas().TestCompatibilityWithSchemaInfo(topic, schemaInfo) + assert.NoError(t, err) + assert.Equal(t, compatibility.IsCompatibility, true) + assert.Equal(t, compatibility.SchemaCompatibilityStrategy, utils.SchemaCompatibilityStrategy("FULL")) + + compatibility, err = admin.Schemas().TestCompatibilityWithPostSchemaPayload(topic, schemaPayload) + assert.NoError(t, err) + assert.Equal(t, compatibility.IsCompatibility, true) + assert.Equal(t, compatibility.SchemaCompatibilityStrategy, utils.SchemaCompatibilityStrategy("FULL")) + + err = admin.Schemas().ForceDeleteSchema(topic) + assert.NoError(t, err) + + _, err = admin.Schemas().GetSchemaInfo(topic) + assert.Errorf(t, err, "Schema not found") + +} diff --git a/pulsaradmin/pkg/utils/schema_util.go b/pulsaradmin/pkg/utils/schema_util.go index 08aaf54ac6..3b83669049 100644 --- a/pulsaradmin/pkg/utils/schema_util.go +++ b/pulsaradmin/pkg/utils/schema_util.go @@ -44,6 +44,11 @@ type GetSchemaResponse struct { Properties map[string]string `json:"properties"` } +type IsCompatibility struct { + IsCompatibility bool `json:"compatibility"` + SchemaCompatibilityStrategy SchemaCompatibilityStrategy `json:"schemaCompatibilityStrategy"` +} + func ConvertGetSchemaResponseToSchemaInfo(tn *TopicName, response GetSchemaResponse) *SchemaInfo { info := new(SchemaInfo) schema := make([]byte, 0, 10) @@ -61,6 +66,24 @@ func ConvertGetSchemaResponseToSchemaInfo(tn *TopicName, response GetSchemaRespo return info } +func ConvertSchemaDataToStringLegacy(schemaInfo SchemaInfo) string { + schema := schemaInfo.Schema + if schema == nil { + return "" + } + // TODO: KEY_VALUE + return string(schema) + +} + +func ConvertSchemaInfoToPostSchemaPayload(schemaInfo SchemaInfo) PostSchemaPayload { + return PostSchemaPayload{ + SchemaType: schemaInfo.Type, + Schema: ConvertSchemaDataToStringLegacy(schemaInfo), + Properties: schemaInfo.Properties, + } +} + func ConvertGetSchemaResponseToSchemaInfoWithVersion(tn *TopicName, response GetSchemaResponse) *SchemaInfoWithVersion { info := new(SchemaInfoWithVersion) info.SchemaInfo = ConvertGetSchemaResponseToSchemaInfo(tn, response) From 8e9087339d0d004231d3f8fc5f823e561840182b Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Tue, 14 May 2024 17:20:31 +0800 Subject: [PATCH 345/348] [fix] Return an error when AckCumulative on a Shared/KeyShared subscription (#1217) ### Motivation The consumer should return error when AckCumulative on a Shared/KeyShared subscription --- pulsar/consumer_partition.go | 14 +++++++++++++- pulsar/consumer_test.go | 35 +++++++++++++++++++++++++++++++++++ 2 files changed, 48 insertions(+), 1 deletion(-) diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index dc01e69285..f752afbc61 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -20,7 +20,6 @@ package pulsar import ( "container/list" "encoding/hex" - "errors" "fmt" "math" "strings" @@ -36,6 +35,7 @@ import ( pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto" "github.com/apache/pulsar-client-go/pulsar/log" "github.com/bits-and-blooms/bitset" + "github.com/pkg/errors" uAtomic "go.uber.org/atomic" ) @@ -50,6 +50,10 @@ const ( consumerClosed ) +var ( + ErrInvalidAck = errors.New("invalid ack") +) + func (s consumerState) String() string { switch s { case consumerInit: @@ -686,12 +690,20 @@ func (pc *partitionConsumer) AckIDWithResponseCumulative(msgID MessageID) error return pc.internalAckIDCumulative(msgID, true) } +func (pc *partitionConsumer) isAllowAckCumulative() bool { + return pc.options.subscriptionType != Shared && pc.options.subscriptionType != KeyShared +} + func (pc *partitionConsumer) internalAckIDCumulative(msgID MessageID, withResponse bool) error { if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing { pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer") return errors.New("consumer state is closed") } + if !pc.isAllowAckCumulative() { + return errors.Wrap(ErrInvalidAck, "cumulative ack is not allowed for the Shared/KeyShared subscription type") + } + // chunk message id will be converted to tracking message id trackingID := toTrackingMessageID(msgID) if trackingID == nil { diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 4120ba4bac..00f48cae51 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -4392,3 +4392,38 @@ func TestMultiConsumerMemoryLimit(t *testing.T) { return assert.Equal(t, pc2PrevQueueSize/2, pc2.currentQueueSize.Load()) }) } + +func TestConsumerAckCumulativeOnSharedSubShouldFailed(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + assert.Nil(t, err) + defer client.Close() + + topic := newTopicName() + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + Type: Shared, + }) + assert.Nil(t, err) + defer consumer.Close() + + producer, err := client.CreateProducer(ProducerOptions{ + Topic: topic, + }) + assert.Nil(t, err) + defer producer.Close() + + _, err = producer.Send(context.Background(), &ProducerMessage{ + Payload: []byte("hello"), + }) + assert.Nil(t, err) + + msg, err := consumer.Receive(context.Background()) + assert.Nil(t, err) + + err = consumer.AckIDCumulative(msg.ID()) + assert.NotNil(t, err) + assert.ErrorIs(t, err, ErrInvalidAck) +} From 43d8cfc414aff426ae9cc74e905d280332c86e3c Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Thu, 16 May 2024 20:25:04 +0800 Subject: [PATCH 346/348] [cleanup] Remove AvroCodec from JSONSchema (#1216) ### Modifications `AvroCodec` is not used. Remove `AvroCodec` from `JSONSchema`. --- pulsar/schema.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar/schema.go b/pulsar/schema.go index 3427fb263a..ac2192dbe9 100644 --- a/pulsar/schema.go +++ b/pulsar/schema.go @@ -142,7 +142,6 @@ func initAvroCodec(codec string) (*goavro.Codec, error) { } type JSONSchema struct { - AvroCodec SchemaInfo } From a09ae3225334fb9587e5087e3adb2f944e89635d Mon Sep 17 00:00:00 2001 From: Gaylor Bosson Date: Mon, 20 May 2024 12:43:04 +0200 Subject: [PATCH 347/348] [Issue 1218][Reader] Reader Next returns on closed consumer (#1219) Fixes #1218 ### Motivation Calling Next on a reader already closed will block forever unless the context is canceled. Similarly, the call will not return if a different go routine closes the reader. ### Modifications Next now listens for the close channel of the consumer to return an error when it closes. Signed-off-by: Gaylor Bosson --- pulsar/reader_impl.go | 2 ++ pulsar/reader_test.go | 24 ++++++++++++++++++++++++ pulsar/table_view_impl.go | 11 ++++++++--- 3 files changed, 34 insertions(+), 3 deletions(-) diff --git a/pulsar/reader_impl.go b/pulsar/reader_impl.go index bf91c67fa5..4f4e2aa993 100644 --- a/pulsar/reader_impl.go +++ b/pulsar/reader_impl.go @@ -171,6 +171,8 @@ func (r *reader) Next(ctx context.Context) (Message, error) { return nil, err } return cm.Message, nil + case <-r.c.closeCh: + return nil, newError(ConsumerClosed, "consumer closed") case <-ctx.Done(): return nil, ctx.Err() } diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index a9c45ba806..d00346fc7c 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -1035,3 +1035,27 @@ func TestReaderHasNextRetryFailed(t *testing.T) { } } + +func TestReaderNextReturnsOnClosedConsumer(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: serviceURL, + OperationTimeout: 2 * time.Second, + }) + assert.NoError(t, err) + topic := newTopicName() + reader, err := client.CreateReader(ReaderOptions{ + Topic: topic, + StartMessageID: EarliestMessageID(), + }) + assert.Nil(t, err) + + reader.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + var e *Error + _, err = reader.Next(ctx) + assert.ErrorAs(t, err, &e) + assert.Equal(t, ConsumerClosed, e.Result()) +} diff --git a/pulsar/table_view_impl.go b/pulsar/table_view_impl.go index 17e0b90f3b..60b66e33d0 100644 --- a/pulsar/table_view_impl.go +++ b/pulsar/table_view_impl.go @@ -129,7 +129,9 @@ func (tv *TableViewImpl) partitionUpdateCheck() error { if err != nil { tv.logger.Errorf("read next message failed for %s: %w", partition, err) } - tv.handleMessage(msg) + if msg != nil { + tv.handleMessage(msg) + } } ctx, cancelFunc := context.WithCancel(context.Background()) tv.cancelRaders[partition] = cancelReader{ @@ -268,9 +270,12 @@ func (tv *TableViewImpl) watchReaderForNewMessages(ctx context.Context, reader R if err != nil { tv.logger.Errorf("read next message failed for %s: %w", reader.Topic(), err) } - if errors.Is(err, context.Canceled) { + var e *Error + if (errors.As(err, &e) && e.Result() == ConsumerClosed) || errors.Is(err, context.Canceled) { return } - tv.handleMessage(msg) + if msg != nil { + tv.handleMessage(msg) + } } } From b0111a2dc473317e4b41ab4af09e42b09a72384c Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Mon, 20 May 2024 18:59:31 +0800 Subject: [PATCH 348/348] [Improve] PIP-313 Support force unsubscribe using consumer api (#1220) ### Motivation To keep consistent with the Java client. Releted PR: /~https://github.com/apache/pulsar/pull/21687 ### Modifications - Add `UnsubscribeForce`api for consumer. --- pulsar/consumer.go | 9 + pulsar/consumer_impl.go | 10 +- pulsar/consumer_multitopic.go | 12 + pulsar/consumer_multitopic_test.go | 53 + pulsar/consumer_partition.go | 8 +- pulsar/consumer_regex.go | 15 + pulsar/consumer_regex_test.go | 10 + pulsar/consumer_test.go | 51 + pulsar/internal/pulsar_proto/PulsarApi.pb.go | 1727 +++++++++-------- pulsar/internal/pulsar_proto/PulsarApi.proto | 1 + .../consumer_interceptor_test.go | 3 + 11 files changed, 1039 insertions(+), 860 deletions(-) diff --git a/pulsar/consumer.go b/pulsar/consumer.go index fea94cf6a3..3161af5143 100644 --- a/pulsar/consumer.go +++ b/pulsar/consumer.go @@ -269,6 +269,15 @@ type Consumer interface { // where more than one consumer are currently connected. Unsubscribe() error + // UnsubscribeForce the consumer, forcefully unsubscribe by disconnecting connected consumers. + // + // Unsubscribing will cause the subscription to be deleted, + // and all the retained data can potentially be deleted based on message retention and ttl policy. + // + // This operation will fail when performed on a shared subscription + // where more than one consumer are currently connected. + UnsubscribeForce() error + // Receive a single message. // This calls blocks until a message is available. Receive(context.Context) (Message, error) diff --git a/pulsar/consumer_impl.go b/pulsar/consumer_impl.go index 0c31a1aafc..b22eeaafce 100644 --- a/pulsar/consumer_impl.go +++ b/pulsar/consumer_impl.go @@ -449,12 +449,20 @@ func (c *consumer) Subscription() string { } func (c *consumer) Unsubscribe() error { + return c.unsubscribe(false) +} + +func (c *consumer) UnsubscribeForce() error { + return c.unsubscribe(true) +} + +func (c *consumer) unsubscribe(force bool) error { c.Lock() defer c.Unlock() var errMsg string for _, consumer := range c.consumers { - if err := consumer.Unsubscribe(); err != nil { + if err := consumer.unsubscribe(force); err != nil { errMsg += fmt.Sprintf("topic %s, subscription %s: %s", consumer.topic, c.Subscription(), err) } } diff --git a/pulsar/consumer_multitopic.go b/pulsar/consumer_multitopic.go index f6630dd65f..eaf42ad748 100644 --- a/pulsar/consumer_multitopic.go +++ b/pulsar/consumer_multitopic.go @@ -97,6 +97,18 @@ func (c *multiTopicConsumer) Unsubscribe() error { return errs } +func (c *multiTopicConsumer) UnsubscribeForce() error { + var errs error + for t, consumer := range c.consumers { + if err := consumer.UnsubscribeForce(); err != nil { + msg := fmt.Sprintf("unable to force unsubscribe from topic=%s subscription=%s", + t, c.Subscription()) + errs = pkgerrors.Wrap(err, msg) + } + } + return errs +} + func (c *multiTopicConsumer) Receive(ctx context.Context) (message Message, err error) { for { select { diff --git a/pulsar/consumer_multitopic_test.go b/pulsar/consumer_multitopic_test.go index 47704f9f73..58ad09576c 100644 --- a/pulsar/consumer_multitopic_test.go +++ b/pulsar/consumer_multitopic_test.go @@ -84,3 +84,56 @@ func TestMultiTopicConsumerReceive(t *testing.T) { } assert.Equal(t, receivedTopic1, receivedTopic2) } + +func TestMultiTopicConsumerUnsubscribe(t *testing.T) { + topic1 := newTopicName() + topic2 := newTopicName() + + client, err := NewClient(ClientOptions{ + URL: "pulsar://localhost:6650", + }) + if err != nil { + t.Fatal(err) + } + topics := []string{topic1, topic2} + consumer, err := client.Subscribe(ConsumerOptions{ + Topics: topics, + SubscriptionName: "multi-topic-sub", + }) + if err != nil { + t.Fatal(err) + } + defer consumer.Close() + + err = consumer.Unsubscribe() + assert.Nil(t, err) + + err = consumer.Unsubscribe() + assert.Error(t, err) + +} +func TestMultiTopicConsumerForceUnsubscribe(t *testing.T) { + topic1 := newTopicName() + topic2 := newTopicName() + + client, err := NewClient(ClientOptions{ + URL: "pulsar://localhost:6650", + }) + if err != nil { + t.Fatal(err) + } + topics := []string{topic1, topic2} + consumer, err := client.Subscribe(ConsumerOptions{ + Topics: topics, + SubscriptionName: "multi-topic-sub", + }) + if err != nil { + t.Fatal(err) + } + defer consumer.Close() + err = consumer.UnsubscribeForce() + assert.Nil(t, err) + + err = consumer.UnsubscribeForce() + assert.Error(t, err) +} diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index f752afbc61..217edd849f 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -411,13 +411,13 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon return pc, nil } -func (pc *partitionConsumer) Unsubscribe() error { +func (pc *partitionConsumer) unsubscribe(force bool) error { if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing { pc.log.WithField("state", state).Error("Failed to unsubscribe closing or closed consumer") - return nil + return errors.New("consumer state is closed") } - req := &unsubscribeRequest{doneCh: make(chan struct{})} + req := &unsubscribeRequest{doneCh: make(chan struct{}), force: force} pc.eventsCh <- req // wait for the request to complete @@ -553,6 +553,7 @@ func (pc *partitionConsumer) internalUnsubscribe(unsub *unsubscribeRequest) { cmdUnsubscribe := &pb.CommandUnsubscribe{ RequestId: proto.Uint64(requestID), ConsumerId: proto.Uint64(pc.consumerID), + Force: proto.Bool(unsub.force), } _, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), requestID, pb.BaseCommand_UNSUBSCRIBE, cmdUnsubscribe) if err != nil { @@ -1544,6 +1545,7 @@ type ackWithTxnRequest struct { type unsubscribeRequest struct { doneCh chan struct{} + force bool err error } diff --git a/pulsar/consumer_regex.go b/pulsar/consumer_regex.go index d36694ef90..87acfe3ea9 100644 --- a/pulsar/consumer_regex.go +++ b/pulsar/consumer_regex.go @@ -133,6 +133,21 @@ func (c *regexConsumer) Unsubscribe() error { return errs } +func (c *regexConsumer) UnsubscribeForce() error { + var errs error + c.consumersLock.Lock() + defer c.consumersLock.Unlock() + + for topic, consumer := range c.consumers { + if err := consumer.UnsubscribeForce(); err != nil { + msg := fmt.Sprintf("unable to force unsubscribe from topic=%s subscription=%s", + topic, c.Subscription()) + errs = pkgerrors.Wrap(err, msg) + } + } + return errs +} + func (c *regexConsumer) Receive(ctx context.Context) (message Message, err error) { for { select { diff --git a/pulsar/consumer_regex_test.go b/pulsar/consumer_regex_test.go index ebd7e4e196..6bc3f7006c 100644 --- a/pulsar/consumer_regex_test.go +++ b/pulsar/consumer_regex_test.go @@ -303,6 +303,11 @@ func runRegexConsumerMatchOneTopic(t *testing.T, c Client, namespace string) { "message does not start with foo: %s", string(m.Payload())) } } + err = consumer.Unsubscribe() + assert.Nil(t, err) + + err = consumer.Unsubscribe() + assert.Error(t, err) } func runRegexConsumerAddMatchingTopic(t *testing.T, c Client, namespace string) { @@ -347,6 +352,11 @@ func runRegexConsumerAddMatchingTopic(t *testing.T, c Client, namespace string) "message does not start with foo: %s", string(m.Payload())) } } + err = consumer.UnsubscribeForce() + assert.Nil(t, err) + + err = consumer.UnsubscribeForce() + assert.Error(t, err) } func runRegexConsumerAutoDiscoverTopics(t *testing.T, c Client, namespace string) { diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 00f48cae51..b3b797f1b6 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -4427,3 +4427,54 @@ func TestConsumerAckCumulativeOnSharedSubShouldFailed(t *testing.T) { assert.NotNil(t, err) assert.ErrorIs(t, err, ErrInvalidAck) } + +func TestConsumerUnSubscribe(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + + topic := "my-topic" + // create consumer + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + Type: Exclusive, + }) + assert.Nil(t, err) + defer consumer.Close() + + err = consumer.Unsubscribe() + assert.Nil(t, err) + + err = consumer.Unsubscribe() + assert.Error(t, err) + +} +func TestConsumerForceUnSubscribe(t *testing.T) { + client, err := NewClient(ClientOptions{ + URL: lookupURL, + }) + + assert.Nil(t, err) + defer client.Close() + + topic := "my-topic" + // create consumer + consumer, err := client.Subscribe(ConsumerOptions{ + Topic: topic, + SubscriptionName: "my-sub", + Type: Exclusive, + }) + assert.Nil(t, err) + defer consumer.Close() + + err = consumer.UnsubscribeForce() + assert.Nil(t, err) + + err = consumer.UnsubscribeForce() + assert.Error(t, err) + +} diff --git a/pulsar/internal/pulsar_proto/PulsarApi.pb.go b/pulsar/internal/pulsar_proto/PulsarApi.pb.go index 5c6e0f3ef5..365ca4605e 100644 --- a/pulsar/internal/pulsar_proto/PulsarApi.pb.go +++ b/pulsar/internal/pulsar_proto/PulsarApi.pb.go @@ -18,8 +18,8 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.28.1 -// protoc v3.21.12 +// protoc-gen-go v1.34.1 +// protoc v5.26.1 // source: PulsarApi.proto package proto @@ -3162,8 +3162,8 @@ type CommandLookupTopic struct { 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=originalAuthData" json:"original_auth_data,omitempty"` - OriginalAuthMethod *string `protobuf:"bytes,6,opt,name=original_auth_method,json=originalAuthMethod" json:"original_auth_method,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"` } @@ -4218,8 +4218,14 @@ type CommandUnsubscribe struct { 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"` + Force *bool `protobuf:"varint,3,opt,name=force,def=0" json:"force,omitempty"` } +// Default values for CommandUnsubscribe fields. +const ( + Default_CommandUnsubscribe_Force = bool(false) +) + func (x *CommandUnsubscribe) Reset() { *x = CommandUnsubscribe{} if protoimpl.UnsafeEnabled { @@ -4266,6 +4272,13 @@ func (x *CommandUnsubscribe) GetRequestId() uint64 { return 0 } +func (x *CommandUnsubscribe) GetForce() bool { + if x != nil && x.Force != nil { + return *x.Force + } + return Default_CommandUnsubscribe_Force +} + // Reset an existing consumer to a particular message id type CommandSeek struct { state protoimpl.MessageState @@ -8245,382 +8258,358 @@ var file_PulsarApi_proto_rawDesc = []byte{ 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x26, 0x0a, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x02, 0x28, 0x0d, 0x52, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, 0x22, 0x54, 0x0a, 0x12, 0x43, 0x6f, 0x6d, 0x6d, + 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, 0x22, 0x71, 0x0a, 0x12, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, - 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x22, 0xbb, - 0x01, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x65, 0x6b, 0x12, 0x1f, - 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, - 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, - 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x3a, - 0x0a, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, - 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x5f, 0x74, 0x69, - 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x3b, 0x0a, 0x18, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, 0x6e, - 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0xd4, 0x01, 0x0a, 0x14, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, - 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, - 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, - 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, - 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, - 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x12, 0x40, - 0x0a, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, - 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, 0x73, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, - 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, 0x73, - 0x22, 0xd4, 0x01, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, - 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, - 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, - 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x61, 0x73, 0x73, - 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x55, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x61, 0x73, 0x73, - 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x55, 0x72, 0x6c, 0x12, 0x40, 0x0a, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, + 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1b, + 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, + 0x61, 0x6c, 0x73, 0x65, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x22, 0xbb, 0x01, 0x0a, 0x0b, + 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x65, 0x6b, 0x12, 0x1f, 0x0a, 0x0b, 0x63, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, + 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, + 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x09, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x5f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x75, + 0x62, 0x6c, 0x69, 0x73, 0x68, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x3b, 0x0a, 0x18, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, 0x6e, 0x64, 0x4f, 0x66, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0xd4, 0x01, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, + 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x49, 0x64, + 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, + 0x3a, 0x0a, 0x18, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, + 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x18, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, + 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x12, 0x40, 0x0a, 0x1b, 0x61, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, 0x73, 0x22, 0xd4, 0x01, + 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x43, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, + 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, + 0x72, 0x6c, 0x12, 0x40, 0x0a, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, + 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, - 0x6c, 0x54, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x61, 0x73, 0x73, 0x69, - 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, 0x73, 0x22, 0xae, 0x01, 0x0a, 0x26, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, - 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, - 0x72, 0x49, 0x64, 0x12, 0x3c, 0x0a, 0x0b, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, - 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, - 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, - 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x65, 0x70, - 0x6f, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x22, 0x5d, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, - 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xff, 0x01, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x53, 0x75, 0x63, 0x63, 0x65, - 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, - 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, - 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, - 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, - 0x3a, 0x02, 0x2d, 0x31, 0x52, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, - 0x63, 0x65, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x2b, 0x0a, 0x0e, - 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x64, - 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x61, 0x64, 0x79, 0x22, 0x78, 0x0a, 0x0c, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x18, 0x02, 0x20, 0x02, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, - 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x22, 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x69, - 0x6e, 0x67, 0x22, 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x6f, 0x6e, - 0x67, 0x22, 0x56, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0x98, 0x05, 0x0a, 0x1c, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, - 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, - 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, - 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, - 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, - 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x6d, 0x73, 0x67, 0x52, - 0x61, 0x74, 0x65, 0x4f, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, 0x6d, 0x73, - 0x67, 0x52, 0x61, 0x74, 0x65, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, 0x67, 0x54, - 0x68, 0x72, 0x6f, 0x75, 0x67, 0x68, 0x70, 0x75, 0x74, 0x4f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x01, 0x52, 0x10, 0x6d, 0x73, 0x67, 0x54, 0x68, 0x72, 0x6f, 0x75, 0x67, 0x68, 0x70, 0x75, - 0x74, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x52, - 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, 0x52, 0x10, - 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, - 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, - 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x10, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, - 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x10, - 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, - 0x12, 0x28, 0x0a, 0x0f, 0x75, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x75, 0x6e, 0x61, 0x63, 0x6b, - 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x42, 0x0a, 0x1c, 0x62, 0x6c, - 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4f, 0x6e, 0x55, - 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x1c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, - 0x72, 0x4f, 0x6e, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, 0x12, 0x18, - 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x65, 0x64, 0x53, 0x69, 0x6e, 0x63, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x53, 0x69, 0x6e, 0x63, 0x65, - 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x74, 0x79, 0x70, 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x45, - 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, 0x6d, 0x73, - 0x67, 0x52, 0x61, 0x74, 0x65, 0x45, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x12, 0x1e, 0x0a, 0x0a, - 0x6d, 0x73, 0x67, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x0a, 0x6d, 0x73, 0x67, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x12, 0x26, 0x0a, 0x0e, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x41, 0x63, 0x6b, 0x52, 0x61, 0x74, 0x65, 0x18, 0x10, - 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x41, 0x63, 0x6b, - 0x52, 0x61, 0x74, 0x65, 0x22, 0x59, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, - 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, + 0x6c, 0x54, 0x6c, 0x73, 0x22, 0xae, 0x01, 0x0a, 0x26, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, + 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, - 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, - 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x22, - 0xe5, 0x01, 0x0a, 0x1f, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, - 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x43, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, - 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x5e, 0x0a, 0x1d, 0x63, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, - 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, - 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x1a, 0x63, 0x6f, 0x6e, - 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4d, 0x61, 0x72, 0x6b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, - 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xa7, 0x02, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, - 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x12, 0x3c, 0x0a, 0x0b, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, + 0x74, 0x61, 0x52, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x73, 0x12, 0x25, + 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x45, 0x70, 0x6f, 0x63, 0x68, 0x22, 0x5d, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x4e, 0x0a, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, - 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x4d, 0x6f, - 0x64, 0x65, 0x3a, 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x52, 0x04, - 0x6d, 0x6f, 0x64, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x70, - 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x33, 0x0a, 0x04, - 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, - 0x4e, 0x54, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x4e, 0x4f, 0x4e, 0x5f, 0x50, 0x45, 0x52, 0x53, - 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4c, 0x4c, 0x10, - 0x02, 0x22, 0xc0, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, - 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, - 0x12, 0x21, 0x0a, 0x08, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x74, 0x65, - 0x72, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, - 0x73, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, - 0x48, 0x61, 0x73, 0x68, 0x12, 0x1e, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x07, 0x63, 0x68, 0x61, - 0x6e, 0x67, 0x65, 0x64, 0x22, 0xbb, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1d, - 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, - 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, - 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, - 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, - 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, 0x09, 0x52, - 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x02, - 0x28, 0x09, 0x52, 0x0d, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, - 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, - 0x73, 0x68, 0x22, 0x93, 0x01, 0x0a, 0x1c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, - 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, - 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, - 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, - 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, - 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x9f, 0x01, 0x0a, 0x17, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, - 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x70, 0x69, 0x63, - 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x65, 0x77, 0x54, 0x6f, 0x70, 0x69, - 0x63, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x65, 0x6c, 0x65, - 0x74, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0a, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x5a, 0x0a, 0x1a, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, - 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, - 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, - 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x22, 0x6e, 0x0a, 0x10, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, - 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, - 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xed, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, - 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, - 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, - 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, - 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x7d, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, - 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, - 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x18, 0x03, 0x20, 0x02, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xc7, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, - 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, - 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, - 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, - 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, - 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, - 0x56, 0x0a, 0x1d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, - 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x22, 0xff, 0x01, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, + 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x23, + 0x0a, 0x0d, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x65, 0x71, 0x75, + 0x65, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x3a, 0x02, 0x2d, + 0x31, 0x52, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x49, + 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, + 0x63, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x2b, 0x0a, 0x0e, 0x70, 0x72, 0x6f, + 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, + 0x72, 0x52, 0x65, 0x61, 0x64, 0x79, 0x22, 0x78, 0x0a, 0x0c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, + 0x20, 0x02, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, + 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x22, 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x69, 0x6e, 0x67, 0x22, + 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x6f, 0x6e, 0x67, 0x22, 0x56, + 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0x98, 0x05, 0x0a, 0x1c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, + 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, + 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, + 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, + 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, + 0x4f, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, 0x6d, 0x73, 0x67, 0x52, 0x61, + 0x74, 0x65, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, 0x67, 0x54, 0x68, 0x72, 0x6f, + 0x75, 0x67, 0x68, 0x70, 0x75, 0x74, 0x4f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x01, 0x52, + 0x10, 0x6d, 0x73, 0x67, 0x54, 0x68, 0x72, 0x6f, 0x75, 0x67, 0x68, 0x70, 0x75, 0x74, 0x4f, 0x75, + 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x52, 0x65, 0x64, 0x65, + 0x6c, 0x69, 0x76, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, 0x52, 0x10, 0x6d, 0x73, 0x67, + 0x52, 0x61, 0x74, 0x65, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x2a, 0x0a, 0x10, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x65, + 0x72, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x10, 0x61, 0x76, 0x61, + 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, 0x12, 0x28, 0x0a, + 0x0f, 0x75, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x75, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x42, 0x0a, 0x1c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, + 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4f, 0x6e, 0x55, 0x6e, 0x61, 0x63, + 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x1c, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4f, 0x6e, + 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x61, + 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, + 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x65, 0x64, 0x53, 0x69, 0x6e, 0x63, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x53, 0x69, 0x6e, 0x63, 0x65, 0x12, 0x12, 0x0a, + 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x45, 0x78, 0x70, 0x69, + 0x72, 0x65, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, 0x6d, 0x73, 0x67, 0x52, 0x61, + 0x74, 0x65, 0x45, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x12, 0x1e, 0x0a, 0x0a, 0x6d, 0x73, 0x67, + 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x6d, + 0x73, 0x67, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x12, 0x26, 0x0a, 0x0e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x41, 0x63, 0x6b, 0x52, 0x61, 0x74, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, + 0x01, 0x52, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x41, 0x63, 0x6b, 0x52, 0x61, 0x74, + 0x65, 0x22, 0x59, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, + 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, + 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, + 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, + 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, + 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x22, 0xe5, 0x01, 0x0a, + 0x1f, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x43, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x49, 0x64, 0x12, 0x5e, 0x0a, 0x1d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x5f, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x70, 0x6f, 0x73, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, + 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x1a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x4d, 0x61, 0x72, 0x6b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xa7, 0x02, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x12, 0x4e, 0x0a, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, + 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x4d, 0x6f, 0x64, 0x65, 0x3a, + 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x52, 0x04, 0x6d, 0x6f, 0x64, + 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x70, 0x61, 0x74, 0x74, + 0x65, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x73, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, + 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x33, 0x0a, 0x04, 0x4d, 0x6f, 0x64, + 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x10, + 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x4e, 0x4f, 0x4e, 0x5f, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, 0x54, + 0x45, 0x4e, 0x54, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4c, 0x4c, 0x10, 0x02, 0x22, 0xc0, + 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, + 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x21, 0x0a, + 0x08, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x3a, + 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, + 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, + 0x68, 0x12, 0x1e, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, + 0x64, 0x22, 0xbb, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, + 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, + 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, + 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, + 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x73, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, 0x52, + 0x0d, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x1f, + 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, + 0x93, 0x01, 0x0a, 0x1c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, - 0x16, 0x0a, 0x05, 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x3a, 0x01, - 0x30, 0x52, 0x04, 0x74, 0x63, 0x49, 0x64, 0x22, 0x8a, 0x01, 0x0a, 0x1e, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, - 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, - 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4e, - 0x65, 0x77, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x12, 0x14, + 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, + 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x9f, 0x01, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, + 0x12, 0x1d, 0x0a, 0x0a, 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x65, 0x77, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, + 0x25, 0x0a, 0x0e, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, + 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, + 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x5a, 0x0a, 0x1a, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, + 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x5f, 0x74, 0x74, 0x6c, 0x5f, - 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, - 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x54, 0x74, 0x6c, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, - 0x16, 0x0a, 0x05, 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, - 0x30, 0x52, 0x04, 0x74, 0x63, 0x49, 0x64, 0x22, 0xd9, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, - 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, - 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, - 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, - 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, - 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, - 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x22, 0xb1, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, - 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, + 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, + 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, + 0x72, 0x49, 0x64, 0x22, 0x6e, 0x0a, 0x10, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, + 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, + 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x25, 0x0a, 0x0e, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x22, 0xed, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, + 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, - 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, - 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, - 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, - 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, - 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xe4, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, - 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, - 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, - 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, - 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, - 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, - 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x48, - 0x0a, 0x0c, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x14, - 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x02, 0x28, 0x09, 0x52, 0x05, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x12, 0x22, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd4, 0x01, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, - 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, - 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, - 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, - 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, - 0x3e, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, - 0xe7, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, + 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2c, + 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, + 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x25, 0x0a, 0x0e, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x22, 0x7d, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, + 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, + 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x14, + 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x05, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, + 0x20, 0x02, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x22, 0xc7, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, + 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, - 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, - 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, - 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, - 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, - 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, - 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, - 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, - 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, - 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x0d, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, - 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, - 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, - 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, - 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, - 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, - 0x74, 0x73, 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x09, 0x74, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd9, 0x01, 0x0a, 0x15, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, + 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, + 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, + 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, + 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x56, 0x0a, 0x1d, + 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, + 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, + 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x05, + 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x04, + 0x74, 0x63, 0x49, 0x64, 0x22, 0x8a, 0x01, 0x0a, 0x1e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, + 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, + 0x64, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x5f, 0x74, 0x74, 0x6c, 0x5f, 0x73, 0x65, 0x63, + 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, + 0x78, 0x6e, 0x54, 0x74, 0x6c, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x16, 0x0a, 0x05, + 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x04, + 0x74, 0x63, 0x49, 0x64, 0x22, 0xd9, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, + 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, + 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, + 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, + 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, + 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, + 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, + 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x22, 0xb1, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, + 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, + 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, + 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, + 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, + 0x42, 0x69, 0x74, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xe4, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, + 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, + 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, + 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, + 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, + 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, + 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x48, 0x0a, 0x0c, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x02, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, + 0x63, 0x12, 0x22, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd4, 0x01, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, + 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, + 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, + 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, + 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, + 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x3e, 0x0a, 0x0c, + 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, + 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xe7, 0x01, 0x0a, + 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, @@ -8633,42 +8622,8 @@ var file_PulsarApi_proto_rawDesc = []byte{ 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xa8, 0x02, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, - 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, - 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, - 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, - 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, - 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x74, - 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, 0x6e, 0x69, - 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, 0x66, 0x5f, - 0x6c, 0x6f, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x06, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, - 0x69, 0x74, 0x73, 0x4f, 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, - 0x6b, 0x22, 0xe4, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, - 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, - 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, - 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, - 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, - 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, - 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, - 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, - 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, - 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, - 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xd5, 0x02, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x0d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, @@ -8676,478 +8631,538 @@ var file_PulsarApi_proto_rawDesc = []byte{ 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, - 0x3e, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, + 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x74, 0x78, - 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, 0x6e, 0x69, 0x64, - 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, 0x66, 0x5f, 0x6c, - 0x6f, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, - 0x74, 0x73, 0x4f, 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, - 0x22, 0xe7, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, - 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, - 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, - 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, - 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, - 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, - 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, - 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, - 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, - 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x9a, 0x2e, 0x0a, 0x0b, 0x42, - 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x32, 0x0a, 0x04, 0x74, 0x79, - 0x70, 0x65, 0x18, 0x01, 0x20, 0x02, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x42, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x36, - 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x07, 0x63, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x12, 0x3c, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x65, 0x64, 0x12, 0x3c, 0x0a, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, - 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x62, 0x65, 0x12, 0x39, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x65, 0x72, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, 0x2d, 0x0a, - 0x04, 0x73, 0x65, 0x6e, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x04, 0x73, 0x65, 0x6e, 0x64, 0x12, 0x43, 0x0a, 0x0c, - 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x18, 0x07, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x63, - 0x65, 0x69, 0x70, 0x74, 0x52, 0x0b, 0x73, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x63, 0x65, 0x69, 0x70, - 0x74, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, - 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, - 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x73, 0x65, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, - 0x12, 0x36, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, - 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2a, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, - 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x63, 0x6b, 0x52, - 0x03, 0x61, 0x63, 0x6b, 0x12, 0x2d, 0x0a, 0x04, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x0b, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x04, 0x66, - 0x6c, 0x6f, 0x77, 0x12, 0x42, 0x0a, 0x0b, 0x75, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x62, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x55, - 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x0b, 0x75, 0x6e, 0x73, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x36, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, - 0x73, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd9, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, + 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, + 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, + 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, + 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, + 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0xa8, 0x02, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, + 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, + 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, + 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, + 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, + 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, + 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x36, 0x0a, + 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x74, 0x78, 0x6e, 0x41, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, + 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, 0x66, 0x5f, 0x6c, 0x6f, 0x77, + 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, + 0x4f, 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x22, 0xe4, + 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, + 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, + 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, + 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, + 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, + 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, + 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, + 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xd5, 0x02, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, + 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, + 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, + 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, + 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, + 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x3e, 0x0a, 0x0c, + 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, + 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x36, 0x0a, 0x0a, + 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x74, 0x78, 0x6e, 0x41, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, + 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, 0x66, 0x5f, 0x6c, 0x6f, 0x77, 0x5f, + 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x4f, + 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x22, 0xe7, 0x01, + 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, + 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, + 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, + 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, + 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, + 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, + 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, + 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, + 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x9a, 0x2e, 0x0a, 0x0b, 0x42, 0x61, 0x73, 0x65, + 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x32, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x01, 0x20, 0x02, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x42, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x36, 0x0a, 0x07, 0x63, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, + 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x12, 0x3c, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, + 0x64, 0x12, 0x3c, 0x0a, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x52, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, + 0x39, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, + 0x52, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, 0x2d, 0x0a, 0x04, 0x73, 0x65, + 0x6e, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, - 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, - 0x30, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x65, 0x6e, 0x64, 0x52, 0x04, 0x73, 0x65, 0x6e, 0x64, 0x12, 0x43, 0x0a, 0x0c, 0x73, 0x65, 0x6e, + 0x64, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x63, 0x65, 0x69, 0x70, + 0x74, 0x52, 0x0b, 0x73, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x12, 0x3d, + 0x0a, 0x0a, 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x45, 0x72, 0x72, + 0x6f, 0x72, 0x52, 0x09, 0x73, 0x65, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x36, 0x0a, + 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x12, 0x49, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x65, 0x72, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x0d, 0x63, - 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x0e, - 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x18, 0x10, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, - 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x43, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x12, 0x4f, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x65, 0x72, 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, - 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, - 0x72, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x69, 0x6e, 0x67, - 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x69, 0x6e, - 0x67, 0x52, 0x04, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x6f, 0x6e, 0x67, 0x18, - 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x6f, 0x6e, 0x67, - 0x52, 0x04, 0x70, 0x6f, 0x6e, 0x67, 0x12, 0x7e, 0x0a, 0x1f, 0x72, 0x65, 0x64, 0x65, 0x6c, 0x69, - 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, - 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x34, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, - 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x1f, 0x72, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x07, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2a, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x0a, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x63, 0x6b, 0x52, 0x03, 0x61, 0x63, + 0x6b, 0x12, 0x2d, 0x0a, 0x04, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x04, 0x66, 0x6c, 0x6f, 0x77, + 0x12, 0x42, 0x0a, 0x0b, 0x75, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x18, + 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x55, 0x6e, 0x73, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x0b, 0x75, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x12, 0x36, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, + 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x75, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x30, 0x0a, 0x05, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, + 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x49, + 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, + 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, + 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, + 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, + 0x73, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x18, 0x10, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x43, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x12, 0x4f, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, + 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, + 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x53, 0x75, 0x63, + 0x63, 0x65, 0x73, 0x73, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x53, 0x75, + 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x12, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x04, + 0x70, 0x69, 0x6e, 0x67, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x6f, 0x6e, 0x67, 0x18, 0x13, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x6f, 0x6e, 0x67, 0x52, 0x04, 0x70, + 0x6f, 0x6e, 0x67, 0x12, 0x7e, 0x0a, 0x1f, 0x72, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x5b, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x15, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x2d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x52, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, - 0x61, 0x74, 0x61, 0x12, 0x73, 0x0a, 0x19, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, 0x65, 0x74, - 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x70, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x70, + 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, + 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x73, 0x52, 0x1f, 0x72, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, + 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x73, 0x12, 0x5b, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, + 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x6c, 0x6f, 0x6f, 0x6b, - 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, - 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, - 0x0b, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x5a, 0x0a, 0x13, - 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x52, 0x13, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0d, 0x63, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, - 0x61, 0x74, 0x73, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, - 0x74, 0x73, 0x12, 0x60, 0x0a, 0x15, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, - 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1a, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x2a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x15, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, - 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, 0x6e, 0x64, - 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, - 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2d, 0x0a, 0x04, 0x73, 0x65, - 0x65, 0x6b, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, - 0x65, 0x65, 0x6b, 0x52, 0x04, 0x73, 0x65, 0x65, 0x6b, 0x12, 0x51, 0x0a, 0x10, 0x67, 0x65, 0x74, - 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x18, 0x1d, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, - 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x10, 0x67, 0x65, 0x74, 0x4c, - 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x69, 0x0a, 0x18, - 0x67, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, + 0x12, 0x73, 0x0a, 0x19, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x16, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, + 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x0b, 0x6c, 0x6f, + 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x5a, 0x0a, 0x13, 0x6c, 0x6f, 0x6f, + 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4c, 0x6f, 0x6f, + 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x52, 0x13, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, + 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, + 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, + 0x60, 0x0a, 0x15, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x18, 0x67, - 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5f, 0x0a, 0x16, 0x61, 0x63, 0x74, 0x69, 0x76, - 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, - 0x65, 0x18, 0x1f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x63, - 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, - 0x67, 0x65, 0x52, 0x14, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, - 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x67, 0x65, 0x74, 0x54, - 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, - 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x52, 0x14, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, - 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x75, 0x0a, 0x1c, 0x67, 0x65, 0x74, 0x54, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, - 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, - 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x52, 0x1c, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, - 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3c, - 0x0a, 0x09, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x22, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x09, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x54, 0x0a, 0x11, - 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x18, 0x23, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, - 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, - 0x11, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0d, 0x61, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, - 0x6e, 0x67, 0x65, 0x18, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x41, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x52, 0x0d, 0x61, - 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x12, 0x45, 0x0a, 0x0c, - 0x61, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x25, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0c, 0x61, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x61, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, - 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x67, 0x65, 0x74, 0x4f, 0x72, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x27, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x67, 0x65, 0x74, 0x4f, - 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x6c, 0x0a, - 0x19, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x28, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x52, 0x19, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x6e, - 0x65, 0x77, 0x54, 0x78, 0x6e, 0x18, 0x32, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x15, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x54, 0x0a, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, 0x6e, 0x64, 0x4f, + 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, + 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x52, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x52, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, 0x6e, 0x64, + 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2d, 0x0a, 0x04, 0x73, 0x65, 0x65, 0x6b, 0x18, + 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x65, 0x6b, + 0x52, 0x04, 0x73, 0x65, 0x65, 0x6b, 0x12, 0x51, 0x0a, 0x10, 0x67, 0x65, 0x74, 0x4c, 0x61, 0x73, + 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x25, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x10, 0x67, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x69, 0x0a, 0x18, 0x67, 0x65, 0x74, + 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x06, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, - 0x12, 0x4b, 0x0a, 0x0e, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x18, 0x33, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x18, 0x67, 0x65, 0x74, 0x4c, + 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5f, 0x0a, 0x16, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x1f, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, + 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x52, + 0x14, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x43, + 0x68, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x20, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, + 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x14, + 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x75, 0x0a, 0x1c, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, + 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x67, + 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3c, 0x0a, 0x09, 0x67, + 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x22, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, + 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x09, + 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x54, 0x0a, 0x11, 0x67, 0x65, 0x74, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x23, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x11, 0x67, 0x65, + 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x48, 0x0a, 0x0d, 0x61, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, + 0x18, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x75, 0x74, + 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x52, 0x0d, 0x61, 0x75, 0x74, 0x68, + 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x12, 0x45, 0x0a, 0x0c, 0x61, 0x75, 0x74, + 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x25, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x21, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x52, 0x0c, 0x61, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x42, 0x0a, 0x0b, 0x61, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, + 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x63, 0x6b, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x27, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x6c, 0x0a, 0x19, 0x67, 0x65, + 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x28, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, + 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x67, + 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x6e, 0x65, 0x77, 0x54, + 0x78, 0x6e, 0x18, 0x32, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4e, - 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0e, 0x6e, - 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, - 0x11, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, - 0x78, 0x6e, 0x18, 0x34, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, + 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x06, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x12, 0x4b, 0x0a, + 0x0e, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, + 0x33, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, + 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0e, 0x6e, 0x65, 0x77, 0x54, + 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x61, 0x64, + 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x18, + 0x34, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x11, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, - 0x52, 0x11, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, - 0x54, 0x78, 0x6e, 0x12, 0x6c, 0x0a, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x18, 0x35, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x18, 0x36, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x14, 0x61, 0x64, 0x64, 0x53, - 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, - 0x12, 0x75, 0x0a, 0x1c, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x18, 0x37, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, - 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x61, 0x64, 0x64, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x65, 0x6e, 0x64, 0x54, 0x78, - 0x6e, 0x18, 0x38, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, - 0x64, 0x54, 0x78, 0x6e, 0x52, 0x06, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x4b, 0x0a, 0x0e, - 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x39, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0e, 0x65, 0x6e, 0x64, 0x54, 0x78, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x65, 0x6e, 0x64, - 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x3a, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x12, 0x6c, 0x0a, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x35, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x52, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, + 0x0a, 0x14, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x18, 0x36, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, + 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x14, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x12, 0x75, 0x0a, + 0x1c, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x37, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x18, 0x38, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, - 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x65, 0x6e, - 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x6c, 0x0a, 0x19, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3b, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, - 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x52, 0x19, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, - 0x14, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x3c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x14, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, - 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x75, 0x0a, 0x1c, - 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3d, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, - 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, - 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x63, 0x0a, 0x16, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x3e, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, - 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x52, 0x16, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x66, 0x0a, 0x17, 0x74, 0x63, 0x43, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x18, 0x3f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x17, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, + 0x6e, 0x52, 0x06, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x4b, 0x0a, 0x0e, 0x65, 0x6e, 0x64, + 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x39, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0e, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, + 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x3a, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x65, 0x6e, 0x64, 0x54, 0x78, + 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x6c, 0x0a, 0x19, + 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3b, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, + 0x19, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x65, 0x6e, + 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x3c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, + 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x14, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x75, 0x0a, 0x1c, 0x65, 0x6e, 0x64, + 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3d, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x52, 0x1c, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x63, 0x0a, 0x16, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x3e, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x16, 0x74, + 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x66, 0x0a, 0x17, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x4b, 0x0a, 0x0e, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, - 0x73, 0x74, 0x18, 0x40, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, - 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x0e, 0x77, - 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x60, 0x0a, - 0x15, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, - 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x41, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x70, + 0x18, 0x3f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, + 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x17, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4b, 0x0a, + 0x0e, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x18, + 0x40, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, + 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x63, + 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x60, 0x0a, 0x15, 0x77, 0x61, + 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x18, 0x41, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, + 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x15, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x51, 0x0a, 0x10, + 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x18, 0x42, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, + 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x10, 0x77, + 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, + 0x5a, 0x0a, 0x13, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, + 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x43, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, - 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x15, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, - 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, - 0x51, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x18, 0x42, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x52, 0x10, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x13, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x43, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x28, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x52, 0x13, 0x77, 0x61, 0x74, 0x63, 0x68, - 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0x86, - 0x0a, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x4f, 0x4e, 0x4e, 0x45, - 0x43, 0x54, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x45, - 0x44, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x42, 0x45, - 0x10, 0x04, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x10, 0x05, - 0x12, 0x08, 0x0a, 0x04, 0x53, 0x45, 0x4e, 0x44, 0x10, 0x06, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x45, - 0x4e, 0x44, 0x5f, 0x52, 0x45, 0x43, 0x45, 0x49, 0x50, 0x54, 0x10, 0x07, 0x12, 0x0e, 0x0a, 0x0a, - 0x53, 0x45, 0x4e, 0x44, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x08, 0x12, 0x0b, 0x0a, 0x07, - 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x43, 0x4b, - 0x10, 0x0a, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x0b, 0x12, 0x0f, 0x0a, 0x0b, - 0x55, 0x4e, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x42, 0x45, 0x10, 0x0c, 0x12, 0x0b, 0x0a, - 0x07, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, 0x53, 0x10, 0x0d, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x52, - 0x52, 0x4f, 0x52, 0x10, 0x0e, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x50, - 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x10, 0x0f, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4c, 0x4f, - 0x53, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x10, 0x10, 0x12, 0x14, 0x0a, - 0x10, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, - 0x53, 0x10, 0x11, 0x12, 0x08, 0x0a, 0x04, 0x50, 0x49, 0x4e, 0x47, 0x10, 0x12, 0x12, 0x08, 0x0a, - 0x04, 0x50, 0x4f, 0x4e, 0x47, 0x10, 0x13, 0x12, 0x25, 0x0a, 0x21, 0x52, 0x45, 0x44, 0x45, 0x4c, - 0x49, 0x56, 0x45, 0x52, 0x5f, 0x55, 0x4e, 0x41, 0x43, 0x4b, 0x4e, 0x4f, 0x57, 0x4c, 0x45, 0x44, - 0x47, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x53, 0x10, 0x14, 0x12, 0x18, - 0x0a, 0x14, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x4d, 0x45, - 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, 0x10, 0x15, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x41, 0x52, 0x54, - 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, - 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x16, 0x12, 0x0a, 0x0a, 0x06, 0x4c, - 0x4f, 0x4f, 0x4b, 0x55, 0x50, 0x10, 0x17, 0x12, 0x13, 0x0a, 0x0f, 0x4c, 0x4f, 0x4f, 0x4b, 0x55, - 0x50, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x18, 0x12, 0x12, 0x0a, 0x0e, - 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x53, 0x10, 0x19, - 0x12, 0x1b, 0x0a, 0x17, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, - 0x54, 0x53, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1a, 0x12, 0x18, 0x0a, - 0x14, 0x52, 0x45, 0x41, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x45, 0x4e, 0x44, 0x5f, 0x4f, 0x46, 0x5f, - 0x54, 0x4f, 0x50, 0x49, 0x43, 0x10, 0x1b, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x45, 0x45, 0x4b, 0x10, - 0x1c, 0x12, 0x17, 0x0a, 0x13, 0x47, 0x45, 0x54, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, 0x4d, 0x45, - 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x1d, 0x12, 0x20, 0x0a, 0x1c, 0x47, 0x45, - 0x54, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x49, - 0x44, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1e, 0x12, 0x1a, 0x0a, 0x16, - 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, - 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x10, 0x1f, 0x12, 0x1b, 0x0a, 0x17, 0x47, 0x45, 0x54, 0x5f, - 0x54, 0x4f, 0x50, 0x49, 0x43, 0x53, 0x5f, 0x4f, 0x46, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, - 0x41, 0x43, 0x45, 0x10, 0x20, 0x12, 0x24, 0x0a, 0x20, 0x47, 0x45, 0x54, 0x5f, 0x54, 0x4f, 0x50, + 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x52, 0x13, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, + 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0x86, 0x0a, 0x0a, 0x04, + 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x10, + 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x45, 0x44, 0x10, 0x03, + 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x42, 0x45, 0x10, 0x04, 0x12, + 0x0c, 0x0a, 0x08, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x10, 0x05, 0x12, 0x08, 0x0a, + 0x04, 0x53, 0x45, 0x4e, 0x44, 0x10, 0x06, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x45, 0x4e, 0x44, 0x5f, + 0x52, 0x45, 0x43, 0x45, 0x49, 0x50, 0x54, 0x10, 0x07, 0x12, 0x0e, 0x0a, 0x0a, 0x53, 0x45, 0x4e, + 0x44, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x08, 0x12, 0x0b, 0x0a, 0x07, 0x4d, 0x45, 0x53, + 0x53, 0x41, 0x47, 0x45, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x43, 0x4b, 0x10, 0x0a, 0x12, + 0x08, 0x0a, 0x04, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x0b, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, + 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x42, 0x45, 0x10, 0x0c, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x55, + 0x43, 0x43, 0x45, 0x53, 0x53, 0x10, 0x0d, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x52, 0x52, 0x4f, 0x52, + 0x10, 0x0e, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x50, 0x52, 0x4f, 0x44, + 0x55, 0x43, 0x45, 0x52, 0x10, 0x0f, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x5f, + 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x10, 0x10, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x52, + 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, 0x53, 0x10, 0x11, + 0x12, 0x08, 0x0a, 0x04, 0x50, 0x49, 0x4e, 0x47, 0x10, 0x12, 0x12, 0x08, 0x0a, 0x04, 0x50, 0x4f, + 0x4e, 0x47, 0x10, 0x13, 0x12, 0x25, 0x0a, 0x21, 0x52, 0x45, 0x44, 0x45, 0x4c, 0x49, 0x56, 0x45, + 0x52, 0x5f, 0x55, 0x4e, 0x41, 0x43, 0x4b, 0x4e, 0x4f, 0x57, 0x4c, 0x45, 0x44, 0x47, 0x45, 0x44, + 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x53, 0x10, 0x14, 0x12, 0x18, 0x0a, 0x14, 0x50, + 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x44, + 0x41, 0x54, 0x41, 0x10, 0x15, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, + 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x52, 0x45, + 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x16, 0x12, 0x0a, 0x0a, 0x06, 0x4c, 0x4f, 0x4f, 0x4b, + 0x55, 0x50, 0x10, 0x17, 0x12, 0x13, 0x0a, 0x0f, 0x4c, 0x4f, 0x4f, 0x4b, 0x55, 0x50, 0x5f, 0x52, + 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x18, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4f, 0x4e, + 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x53, 0x10, 0x19, 0x12, 0x1b, 0x0a, + 0x17, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x53, 0x5f, + 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1a, 0x12, 0x18, 0x0a, 0x14, 0x52, 0x45, + 0x41, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x45, 0x4e, 0x44, 0x5f, 0x4f, 0x46, 0x5f, 0x54, 0x4f, 0x50, + 0x49, 0x43, 0x10, 0x1b, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x45, 0x45, 0x4b, 0x10, 0x1c, 0x12, 0x17, + 0x0a, 0x13, 0x47, 0x45, 0x54, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, + 0x47, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x1d, 0x12, 0x20, 0x0a, 0x1c, 0x47, 0x45, 0x54, 0x5f, 0x4c, + 0x41, 0x53, 0x54, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x49, 0x44, 0x5f, 0x52, + 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1e, 0x12, 0x1a, 0x0a, 0x16, 0x41, 0x43, 0x54, + 0x49, 0x56, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x43, 0x48, 0x41, + 0x4e, 0x47, 0x45, 0x10, 0x1f, 0x12, 0x1b, 0x0a, 0x17, 0x47, 0x45, 0x54, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x53, 0x5f, 0x4f, 0x46, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, - 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x21, 0x12, 0x0e, 0x0a, 0x0a, 0x47, - 0x45, 0x54, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x22, 0x12, 0x17, 0x0a, 0x13, 0x47, - 0x45, 0x54, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, - 0x53, 0x45, 0x10, 0x23, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x55, 0x54, 0x48, 0x5f, 0x43, 0x48, 0x41, - 0x4c, 0x4c, 0x45, 0x4e, 0x47, 0x45, 0x10, 0x24, 0x12, 0x11, 0x0a, 0x0d, 0x41, 0x55, 0x54, 0x48, - 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x25, 0x12, 0x10, 0x0a, 0x0c, 0x41, - 0x43, 0x4b, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x26, 0x12, 0x18, 0x0a, - 0x14, 0x47, 0x45, 0x54, 0x5f, 0x4f, 0x52, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f, 0x53, - 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x27, 0x12, 0x21, 0x0a, 0x1d, 0x47, 0x45, 0x54, 0x5f, 0x4f, - 0x52, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, - 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x28, 0x12, 0x0b, 0x0a, 0x07, 0x4e, 0x45, - 0x57, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x32, 0x12, 0x14, 0x0a, 0x10, 0x4e, 0x45, 0x57, 0x5f, 0x54, - 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x33, 0x12, 0x18, 0x0a, - 0x14, 0x41, 0x44, 0x44, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, - 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x34, 0x12, 0x21, 0x0a, 0x1d, 0x41, 0x44, 0x44, 0x5f, 0x50, - 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x5f, - 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x35, 0x12, 0x1b, 0x0a, 0x17, 0x41, 0x44, - 0x44, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, - 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x36, 0x12, 0x24, 0x0a, 0x20, 0x41, 0x44, 0x44, 0x5f, 0x53, + 0x10, 0x20, 0x12, 0x24, 0x0a, 0x20, 0x47, 0x45, 0x54, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x53, + 0x5f, 0x4f, 0x46, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x5f, 0x52, 0x45, + 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x21, 0x12, 0x0e, 0x0a, 0x0a, 0x47, 0x45, 0x54, 0x5f, + 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x22, 0x12, 0x17, 0x0a, 0x13, 0x47, 0x45, 0x54, 0x5f, + 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, + 0x23, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x55, 0x54, 0x48, 0x5f, 0x43, 0x48, 0x41, 0x4c, 0x4c, 0x45, + 0x4e, 0x47, 0x45, 0x10, 0x24, 0x12, 0x11, 0x0a, 0x0d, 0x41, 0x55, 0x54, 0x48, 0x5f, 0x52, 0x45, + 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x25, 0x12, 0x10, 0x0a, 0x0c, 0x41, 0x43, 0x4b, 0x5f, + 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x26, 0x12, 0x18, 0x0a, 0x14, 0x47, 0x45, + 0x54, 0x5f, 0x4f, 0x52, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, + 0x4d, 0x41, 0x10, 0x27, 0x12, 0x21, 0x0a, 0x1d, 0x47, 0x45, 0x54, 0x5f, 0x4f, 0x52, 0x5f, 0x43, + 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x52, 0x45, 0x53, + 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x28, 0x12, 0x0b, 0x0a, 0x07, 0x4e, 0x45, 0x57, 0x5f, 0x54, + 0x58, 0x4e, 0x10, 0x32, 0x12, 0x14, 0x0a, 0x10, 0x4e, 0x45, 0x57, 0x5f, 0x54, 0x58, 0x4e, 0x5f, + 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x33, 0x12, 0x18, 0x0a, 0x14, 0x41, 0x44, + 0x44, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, 0x5f, 0x54, + 0x58, 0x4e, 0x10, 0x34, 0x12, 0x21, 0x0a, 0x1d, 0x41, 0x44, 0x44, 0x5f, 0x50, 0x41, 0x52, 0x54, + 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, + 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x35, 0x12, 0x1b, 0x0a, 0x17, 0x41, 0x44, 0x44, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, 0x5f, 0x54, - 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x37, 0x12, 0x0b, 0x0a, - 0x07, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x38, 0x12, 0x14, 0x0a, 0x10, 0x45, 0x4e, - 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x39, - 0x12, 0x18, 0x0a, 0x14, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x50, - 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3a, 0x12, 0x21, 0x0a, 0x1d, 0x45, 0x4e, - 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, - 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3b, 0x12, 0x1b, 0x0a, - 0x17, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x55, 0x42, 0x53, - 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3c, 0x12, 0x24, 0x0a, 0x20, 0x45, 0x4e, + 0x58, 0x4e, 0x10, 0x36, 0x12, 0x24, 0x0a, 0x20, 0x41, 0x44, 0x44, 0x5f, 0x53, 0x55, 0x42, 0x53, + 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x5f, + 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x37, 0x12, 0x0b, 0x0a, 0x07, 0x45, 0x4e, + 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x38, 0x12, 0x14, 0x0a, 0x10, 0x45, 0x4e, 0x44, 0x5f, 0x54, + 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x39, 0x12, 0x18, 0x0a, + 0x14, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x50, 0x41, 0x52, 0x54, + 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3a, 0x12, 0x21, 0x0a, 0x1d, 0x45, 0x4e, 0x44, 0x5f, 0x54, + 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3b, 0x12, 0x1b, 0x0a, 0x17, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, - 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3d, - 0x12, 0x1d, 0x0a, 0x19, 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4f, - 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x10, 0x3e, 0x12, - 0x1e, 0x0a, 0x1a, 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4f, 0x4e, - 0x4e, 0x45, 0x43, 0x54, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3f, 0x12, - 0x14, 0x0a, 0x10, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, - 0x49, 0x53, 0x54, 0x10, 0x40, 0x12, 0x1c, 0x0a, 0x18, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, - 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x5f, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, - 0x53, 0x10, 0x41, 0x12, 0x16, 0x0a, 0x12, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, - 0x49, 0x43, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x42, 0x12, 0x1a, 0x0a, 0x16, 0x57, - 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x5f, - 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x10, 0x43, 0x2a, 0x44, 0x0a, 0x0f, 0x43, 0x6f, 0x6d, 0x70, 0x72, - 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x4f, - 0x4e, 0x45, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x4c, 0x5a, 0x34, 0x10, 0x01, 0x12, 0x08, 0x0a, - 0x04, 0x5a, 0x4c, 0x49, 0x42, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x5a, 0x53, 0x54, 0x44, 0x10, - 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x4e, 0x41, 0x50, 0x50, 0x59, 0x10, 0x04, 0x2a, 0x5f, 0x0a, - 0x12, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4d, - 0x6f, 0x64, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x10, 0x00, 0x12, - 0x0d, 0x0a, 0x09, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x10, 0x01, 0x12, 0x14, - 0x0a, 0x10, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, - 0x76, 0x65, 0x10, 0x02, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, - 0x65, 0x57, 0x69, 0x74, 0x68, 0x46, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x2a, 0x82, - 0x05, 0x0a, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x10, - 0x0a, 0x0c, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x00, - 0x12, 0x11, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x72, 0x72, 0x6f, - 0x72, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, - 0x63, 0x65, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x02, 0x12, 0x17, 0x0a, 0x13, 0x41, 0x75, 0x74, - 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, - 0x10, 0x03, 0x12, 0x16, 0x0a, 0x12, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x04, 0x12, 0x10, 0x0a, 0x0c, 0x43, 0x6f, - 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x42, 0x75, 0x73, 0x79, 0x10, 0x05, 0x12, 0x13, 0x0a, 0x0f, - 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x6f, 0x74, 0x52, 0x65, 0x61, 0x64, 0x79, 0x10, - 0x06, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x42, 0x6c, 0x6f, - 0x63, 0x6b, 0x65, 0x64, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x45, 0x78, 0x63, 0x65, 0x65, 0x64, 0x65, - 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x07, 0x12, 0x29, 0x0a, 0x25, 0x50, 0x72, 0x6f, 0x64, - 0x75, 0x63, 0x65, 0x72, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x51, 0x75, 0x6f, 0x74, 0x61, - 0x45, 0x78, 0x63, 0x65, 0x65, 0x64, 0x65, 0x64, 0x45, 0x78, 0x63, 0x65, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x10, 0x08, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x45, - 0x72, 0x72, 0x6f, 0x72, 0x10, 0x09, 0x12, 0x1b, 0x0a, 0x17, 0x55, 0x6e, 0x73, 0x75, 0x70, 0x70, - 0x6f, 0x72, 0x74, 0x65, 0x64, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, - 0x72, 0x10, 0x0a, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x6f, 0x74, 0x46, - 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0b, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0c, - 0x12, 0x14, 0x0a, 0x10, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x6f, 0x74, 0x46, - 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0d, 0x12, 0x13, 0x0a, 0x0f, 0x54, 0x6f, 0x6f, 0x4d, 0x61, 0x6e, - 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x10, 0x0e, 0x12, 0x18, 0x0a, 0x14, 0x54, - 0x6f, 0x70, 0x69, 0x63, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x45, 0x72, - 0x72, 0x6f, 0x72, 0x10, 0x0f, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, - 0x72, 0x42, 0x75, 0x73, 0x79, 0x10, 0x10, 0x12, 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, 0x61, 0x6c, - 0x69, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x11, 0x12, 0x16, 0x0a, - 0x12, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x10, 0x12, 0x12, 0x17, 0x0a, 0x13, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, - 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x13, 0x12, 0x22, - 0x0a, 0x1e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6f, - 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, - 0x10, 0x14, 0x12, 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x54, 0x78, 0x6e, - 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x10, 0x15, 0x12, 0x13, 0x0a, 0x0f, 0x4e, 0x6f, 0x74, 0x41, - 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x16, 0x12, 0x17, 0x0a, - 0x13, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, - 0x6c, 0x69, 0x63, 0x74, 0x10, 0x17, 0x12, 0x17, 0x0a, 0x13, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x18, 0x12, - 0x12, 0x0a, 0x0e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x46, 0x65, 0x6e, 0x63, 0x65, - 0x64, 0x10, 0x19, 0x2a, 0x4b, 0x0a, 0x0a, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, - 0x64, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4e, - 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, - 0x68, 0x6f, 0x64, 0x59, 0x63, 0x61, 0x56, 0x31, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x41, 0x75, - 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x41, 0x74, 0x68, 0x65, 0x6e, 0x73, 0x10, 0x02, - 0x2a, 0xbb, 0x01, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x56, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x30, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, - 0x76, 0x31, 0x10, 0x01, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x32, 0x10, 0x02, 0x12, 0x06, 0x0a, 0x02, - 0x76, 0x33, 0x10, 0x03, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x34, 0x10, 0x04, 0x12, 0x06, 0x0a, 0x02, - 0x76, 0x35, 0x10, 0x05, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x36, 0x10, 0x06, 0x12, 0x06, 0x0a, 0x02, - 0x76, 0x37, 0x10, 0x07, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x38, 0x10, 0x08, 0x12, 0x06, 0x0a, 0x02, - 0x76, 0x39, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x30, 0x10, 0x0a, 0x12, 0x07, 0x0a, - 0x03, 0x76, 0x31, 0x31, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x32, 0x10, 0x0c, 0x12, - 0x07, 0x0a, 0x03, 0x76, 0x31, 0x33, 0x10, 0x0d, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x34, 0x10, - 0x0e, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x35, 0x10, 0x0f, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, - 0x36, 0x10, 0x10, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x37, 0x10, 0x11, 0x12, 0x07, 0x0a, 0x03, - 0x76, 0x31, 0x38, 0x10, 0x12, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x39, 0x10, 0x13, 0x2a, 0x2b, - 0x0a, 0x0d, 0x4b, 0x65, 0x79, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x4d, 0x6f, 0x64, 0x65, 0x12, - 0x0e, 0x0a, 0x0a, 0x41, 0x55, 0x54, 0x4f, 0x5f, 0x53, 0x50, 0x4c, 0x49, 0x54, 0x10, 0x00, 0x12, - 0x0a, 0x0a, 0x06, 0x53, 0x54, 0x49, 0x43, 0x4b, 0x59, 0x10, 0x01, 0x2a, 0x22, 0x0a, 0x09, 0x54, - 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x4f, 0x4d, 0x4d, - 0x49, 0x54, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x10, 0x01, 0x42, - 0x2f, 0x0a, 0x22, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x61, 0x70, 0x69, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x48, 0x03, 0x5a, 0x07, 0x2e, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3c, 0x12, 0x24, 0x0a, 0x20, 0x45, 0x4e, 0x44, 0x5f, 0x54, + 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3d, 0x12, 0x1d, 0x0a, + 0x19, 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4f, 0x4e, 0x4e, 0x45, + 0x43, 0x54, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x10, 0x3e, 0x12, 0x1e, 0x0a, 0x1a, + 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, + 0x54, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3f, 0x12, 0x14, 0x0a, 0x10, + 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, 0x54, + 0x10, 0x40, 0x12, 0x1c, 0x0a, 0x18, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, + 0x43, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x5f, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, 0x53, 0x10, 0x41, + 0x12, 0x16, 0x0a, 0x12, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, + 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x42, 0x12, 0x1a, 0x0a, 0x16, 0x57, 0x41, 0x54, 0x43, + 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x5f, 0x43, 0x4c, 0x4f, + 0x53, 0x45, 0x10, 0x43, 0x2a, 0x44, 0x0a, 0x0f, 0x43, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, + 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x4f, 0x4e, 0x45, 0x10, + 0x00, 0x12, 0x07, 0x0a, 0x03, 0x4c, 0x5a, 0x34, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x5a, 0x4c, + 0x49, 0x42, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x5a, 0x53, 0x54, 0x44, 0x10, 0x03, 0x12, 0x0a, + 0x0a, 0x06, 0x53, 0x4e, 0x41, 0x50, 0x50, 0x59, 0x10, 0x04, 0x2a, 0x5f, 0x0a, 0x12, 0x50, 0x72, + 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x64, 0x65, + 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, + 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x57, + 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x10, + 0x02, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x57, 0x69, + 0x74, 0x68, 0x46, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x2a, 0x82, 0x05, 0x0a, 0x0b, + 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x10, 0x0a, 0x0c, 0x55, + 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x00, 0x12, 0x11, 0x0a, + 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x01, + 0x12, 0x14, 0x0a, 0x10, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x63, 0x65, 0x45, + 0x72, 0x72, 0x6f, 0x72, 0x10, 0x02, 0x12, 0x17, 0x0a, 0x13, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, + 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x03, 0x12, + 0x16, 0x0a, 0x12, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x04, 0x12, 0x10, 0x0a, 0x0c, 0x43, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x42, 0x75, 0x73, 0x79, 0x10, 0x05, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x4e, 0x6f, 0x74, 0x52, 0x65, 0x61, 0x64, 0x79, 0x10, 0x06, 0x12, 0x25, + 0x0a, 0x21, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x65, + 0x64, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x45, 0x78, 0x63, 0x65, 0x65, 0x64, 0x65, 0x64, 0x45, 0x72, + 0x72, 0x6f, 0x72, 0x10, 0x07, 0x12, 0x29, 0x0a, 0x25, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, + 0x72, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x45, 0x78, 0x63, + 0x65, 0x65, 0x64, 0x65, 0x64, 0x45, 0x78, 0x63, 0x65, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, + 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x45, 0x72, 0x72, 0x6f, + 0x72, 0x10, 0x09, 0x12, 0x1b, 0x0a, 0x17, 0x55, 0x6e, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, + 0x65, 0x64, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x0a, + 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, + 0x64, 0x10, 0x0b, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0c, 0x12, 0x14, 0x0a, + 0x10, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, + 0x64, 0x10, 0x0d, 0x12, 0x13, 0x0a, 0x0f, 0x54, 0x6f, 0x6f, 0x4d, 0x61, 0x6e, 0x79, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x10, 0x0e, 0x12, 0x18, 0x0a, 0x14, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x10, 0x0f, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x42, 0x75, + 0x73, 0x79, 0x10, 0x10, 0x12, 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x11, 0x12, 0x16, 0x0a, 0x12, 0x49, 0x6e, + 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x10, 0x12, 0x12, 0x17, 0x0a, 0x13, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x13, 0x12, 0x22, 0x0a, 0x1e, 0x54, + 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, + 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x14, 0x12, + 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x10, 0x15, 0x12, 0x13, 0x0a, 0x0f, 0x4e, 0x6f, 0x74, 0x41, 0x6c, 0x6c, 0x6f, + 0x77, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x16, 0x12, 0x17, 0x0a, 0x13, 0x54, 0x72, + 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, + 0x74, 0x10, 0x17, 0x12, 0x17, 0x0a, 0x13, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x18, 0x12, 0x12, 0x0a, 0x0e, + 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x46, 0x65, 0x6e, 0x63, 0x65, 0x64, 0x10, 0x19, + 0x2a, 0x4b, 0x0a, 0x0a, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x12, + 0x0a, 0x0e, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4e, 0x6f, 0x6e, 0x65, + 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, + 0x59, 0x63, 0x61, 0x56, 0x31, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x41, 0x75, 0x74, 0x68, 0x4d, + 0x65, 0x74, 0x68, 0x6f, 0x64, 0x41, 0x74, 0x68, 0x65, 0x6e, 0x73, 0x10, 0x02, 0x2a, 0xbb, 0x01, + 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x30, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x31, 0x10, + 0x01, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x32, 0x10, 0x02, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x33, 0x10, + 0x03, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x34, 0x10, 0x04, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x35, 0x10, + 0x05, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x36, 0x10, 0x06, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x37, 0x10, + 0x07, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x38, 0x10, 0x08, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x39, 0x10, + 0x09, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x30, 0x10, 0x0a, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, + 0x31, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x32, 0x10, 0x0c, 0x12, 0x07, 0x0a, 0x03, + 0x76, 0x31, 0x33, 0x10, 0x0d, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x34, 0x10, 0x0e, 0x12, 0x07, + 0x0a, 0x03, 0x76, 0x31, 0x35, 0x10, 0x0f, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x36, 0x10, 0x10, + 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x37, 0x10, 0x11, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x38, + 0x10, 0x12, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x39, 0x10, 0x13, 0x2a, 0x2b, 0x0a, 0x0d, 0x4b, + 0x65, 0x79, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0e, 0x0a, 0x0a, + 0x41, 0x55, 0x54, 0x4f, 0x5f, 0x53, 0x50, 0x4c, 0x49, 0x54, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, + 0x53, 0x54, 0x49, 0x43, 0x4b, 0x59, 0x10, 0x01, 0x2a, 0x22, 0x0a, 0x09, 0x54, 0x78, 0x6e, 0x41, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x10, + 0x00, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x10, 0x01, 0x42, 0x2f, 0x0a, 0x22, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x48, 0x03, 0x5a, 0x07, 0x2e, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, } var ( diff --git a/pulsar/internal/pulsar_proto/PulsarApi.proto b/pulsar/internal/pulsar_proto/PulsarApi.proto index a95a114802..3ef4a123e5 100644 --- a/pulsar/internal/pulsar_proto/PulsarApi.proto +++ b/pulsar/internal/pulsar_proto/PulsarApi.proto @@ -604,6 +604,7 @@ message CommandFlow { message CommandUnsubscribe { required uint64 consumer_id = 1; required uint64 request_id = 2; + optional bool force = 3 [default = false]; } // Reset an existing consumer to a particular message id diff --git a/pulsar/internal/pulsartracing/consumer_interceptor_test.go b/pulsar/internal/pulsartracing/consumer_interceptor_test.go index 06c9a588ff..92ce2311c1 100644 --- a/pulsar/internal/pulsartracing/consumer_interceptor_test.go +++ b/pulsar/internal/pulsartracing/consumer_interceptor_test.go @@ -59,6 +59,9 @@ func (c *mockConsumer) AckWithTxn(msg pulsar.Message, txn pulsar.Transaction) er func (c *mockConsumer) Unsubscribe() error { return nil } +func (c *mockConsumer) UnsubscribeForce() error { + return nil +} func (c *mockConsumer) Receive(ctx context.Context) (message pulsar.Message, err error) { return nil, nil