// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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/apache/pulsar-client-go/pulsar/internal/auth" ) func NewClient(options ClientOptions) (Client, error) { return newClient(options) } // Opaque interface that represents the authentication credentials type Authentication interface{} func NewAuthentication(name string, params string) (Authentication, error) { return auth.NewProvider(name, params) } // Create new Authentication provider with specified auth token func NewAuthenticationToken(token string) Authentication { return auth.NewAuthenticationToken(token) } // NewAuthenticationTokenFromSupplier returns a token auth provider that // gets the token data from a user supplied function. The function is // invoked each time the client library needs to use a token in talking // with Pulsar brokers func NewAuthenticationTokenFromSupplier(tokenSupplier func() (string, error)) Authentication { return auth.NewAuthenticationTokenFromSupplier(tokenSupplier) } // Create new Authentication provider with specified auth token from a file func NewAuthenticationTokenFromFile(tokenFilePath string) Authentication { return auth.NewAuthenticationTokenFromFile(tokenFilePath) } // Create new Authentication provider with specified TLS certificate and private key func NewAuthenticationTLS(certificatePath string, privateKeyPath string) Authentication { return auth.NewAuthenticationTLS(certificatePath, privateKeyPath) } func NewAuthenticationAthenz(authParams map[string]string) Authentication { athenz, _ := auth.NewAuthenticationAthenzWithParams(authParams) return athenz } // Builder interface that is used to construct a Pulsar Client instance. type ClientOptions struct { // Configure the service URL for the Pulsar service. // This parameter is required URL string // Timeout for the establishment of a TCP connection (default: 30 seconds) ConnectionTimeout time.Duration // Set the operation timeout (default: 30 seconds) // Producer-create, subscribe and unsubscribe operations will be retried until this interval, after which the // operation will be marked as failed OperationTimeout time.Duration // Configure the authentication provider. (default: no authentication) // Example: `Authentication: NewAuthenticationTLS("my-cert.pem", "my-key.pem")` Authentication // 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 // Configure whether the Pulsar client verify the validity of the host name from broker (default: false) TLSValidateHostname bool } type Client interface { // Create the producer instance // This method will block until the producer is created successfully CreateProducer(ProducerOptions) (Producer, error) // Create a `Consumer` by subscribing to a topic. // // If the subscription does not exist, a new subscription will be created and all messages published after the // creation will be retained until acknowledged, even if the consumer is not connected Subscribe(ConsumerOptions) (Consumer, error) // Create a Reader instance. // This method will block until the reader is created successfully. CreateReader(ReaderOptions) (Reader, error) // Fetch the list of partitions for a given topic // // If the topic is partitioned, this will return a list of partition names. // If the topic is not partitioned, the returned list will contain the topic // name itself. // // This can be used to discover the partitions and create {@link Reader}, // {@link Consumer} or {@link Producer} instances directly on a particular partition. TopicPartitions(topic string) ([]string, error) // Close the Client and free associated resources Close() }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "fmt" "net/url" "time" "github.com/golang/protobuf/proto" log "github.com/sirupsen/logrus" "github.com/apache/pulsar-client-go/pulsar/internal" "github.com/apache/pulsar-client-go/pulsar/internal/auth" "github.com/apache/pulsar-client-go/pulsar/internal/pb" ) const ( defaultConnectionTimeout = 30 * time.Second defaultOperationTimeout = 30 * time.Second ) type client struct { cnxPool internal.ConnectionPool rpcClient internal.RPCClient handlers internal.ClientHandlers lookupService internal.LookupService } func newClient(options ClientOptions) (Client, error) { if options.URL == "" { return nil, newError(ResultInvalidConfiguration, "URL is required for client") } url, err := url.Parse(options.URL) if err != nil { log.WithError(err).Error("Failed to parse service URL") return nil, newError(ResultInvalidConfiguration, "Invalid service URL") } var tlsConfig *internal.TLSOptions switch url.Scheme { case "pulsar": tlsConfig = nil case "pulsar+ssl": tlsConfig = &internal.TLSOptions{ AllowInsecureConnection: options.TLSAllowInsecureConnection, TrustCertsFilePath: options.TLSTrustCertsFilePath, ValidateHostname: options.TLSValidateHostname, } default: return nil, newError(ResultInvalidConfiguration, fmt.Sprintf("Invalid URL scheme '%s'", url.Scheme)) } var authProvider auth.Provider var ok bool if options.Authentication == nil { authProvider = auth.NewAuthDisabled() } else { authProvider, ok = options.Authentication.(auth.Provider) if !ok { return nil, errors.New("invalid auth provider interface") } } err = authProvider.Init() if err != nil { return nil, err } connectionTimeout := options.ConnectionTimeout if connectionTimeout.Nanoseconds() == 0 { connectionTimeout = defaultConnectionTimeout } operationTimeout := options.OperationTimeout if operationTimeout.Nanoseconds() == 0 { operationTimeout = defaultOperationTimeout } c := &client{ cnxPool: internal.NewConnectionPool(tlsConfig, authProvider, connectionTimeout), } c.rpcClient = internal.NewRPCClient(url, c.cnxPool, operationTimeout) c.lookupService = internal.NewLookupService(c.rpcClient, url, tlsConfig != nil) c.handlers = internal.NewClientHandlers() return c, nil } func (c *client) CreateProducer(options ProducerOptions) (Producer, error) { producer, err := newProducer(c, &options) if err == nil { c.handlers.Add(producer) } return producer, err } func (c *client) Subscribe(options ConsumerOptions) (Consumer, error) { consumer, err := newConsumer(c, options) if err != nil { return nil, err } c.handlers.Add(consumer) return consumer, nil } func (c *client) CreateReader(options ReaderOptions) (Reader, error) { reader, err := newReader(c, options) if err != nil { return nil, err } c.handlers.Add(reader) return reader, nil } func (c *client) TopicPartitions(topic string) ([]string, error) { topicName, err := internal.ParseTopicName(topic) if err != nil { return nil, err } id := c.rpcClient.NewRequestID() res, err := c.rpcClient.RequestToAnyBroker(id, pb.BaseCommand_PARTITIONED_METADATA, &pb.CommandPartitionedTopicMetadata{ RequestId: &id, Topic: &topicName.Name, }) if err != nil { return nil, err } r := res.Response.PartitionMetadataResponse if r != nil { if r.Error != nil { return nil, newError(ResultLookupError, r.GetError().String()) } if r.GetPartitions() > 0 { partitions := make([]string, r.GetPartitions()) for i := 0; i < int(r.GetPartitions()); i++ { partitions[i] = fmt.Sprintf("%s-partition-%d", topic, i) } return partitions, nil } } // Non-partitioned topic return []string{topicName.Name}, nil } func (c *client) Close() { c.handlers.Close() } func (c *client) namespaceTopics(namespace string) ([]string, error) { id := c.rpcClient.NewRequestID() req := &pb.CommandGetTopicsOfNamespace{ RequestId: proto.Uint64(id), Namespace: proto.String(namespace), Mode: pb.CommandGetTopicsOfNamespace_PERSISTENT.Enum(), } res, err := c.rpcClient.RequestToAnyBroker(id, pb.BaseCommand_GET_TOPICS_OF_NAMESPACE, req) if err != nil { return nil, err } if res.Response.Error != nil { return []string{}, newError(ResultLookupError, res.Response.GetError().String()) } return res.Response.GetTopicsOfNamespaceResponse.GetTopics(), nil }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "errors" "fmt" "math/rand" "sync" "time" log "github.com/sirupsen/logrus" "github.com/apache/pulsar-client-go/pulsar/internal" "github.com/apache/pulsar-client-go/pulsar/internal/pb" ) var ErrConsumerClosed = errors.New("consumer closed") const defaultNackRedeliveryDelay = 1 * time.Minute type acker interface { AckID(id *messageID) NackID(id *messageID) } type consumer struct { sync.Mutex topic string client *client options ConsumerOptions consumers []*partitionConsumer consumerName string disableForceTopicCreation bool // channel used to deliver message to clients messageCh chan ConsumerMessage dlq *dlqRouter closeOnce sync.Once closeCh chan struct{} errorCh chan error ticker *time.Ticker log *log.Entry } func newConsumer(client *client, options ConsumerOptions) (Consumer, error) { if options.Topic == "" && options.Topics == nil && options.TopicsPattern == "" { return nil, newError(TopicNotFound, "topic is required") } if options.SubscriptionName == "" { return nil, newError(SubscriptionNotFound, "subscription name is required for consumer") } if options.ReceiverQueueSize == 0 { options.ReceiverQueueSize = 1000 } // did the user pass in a message channel? messageCh := options.MessageChannel if options.MessageChannel == nil { messageCh = make(chan ConsumerMessage, 10) } dlq, err := newDlqRouter(client, options.DLQ) if err != nil { return nil, err } // single topic consumer if options.Topic != "" || len(options.Topics) == 1 { topic := options.Topic if topic == "" { topic = options.Topics[0] } if err := validateTopicNames(topic); err != nil { return nil, err } return topicSubscribe(client, options, topic, messageCh, dlq) } if len(options.Topics) > 1 { if err := validateTopicNames(options.Topics...); err != nil { return nil, err } return newMultiTopicConsumer(client, options, options.Topics, messageCh, dlq) } if options.TopicsPattern != "" { tn, err := internal.ParseTopicName(options.TopicsPattern) if err != nil { return nil, err } pattern, err := extractTopicPattern(tn) if err != nil { return nil, err } return newRegexConsumer(client, options, tn, pattern, messageCh, dlq) } return nil, newError(ResultInvalidTopicName, "topic name is required for consumer") } func newInternalConsumer(client *client, options ConsumerOptions, topic string, messageCh chan ConsumerMessage, dlq *dlqRouter, disableForceTopicCreation bool) (*consumer, error) { consumer := &consumer{ topic: topic, client: client, options: options, disableForceTopicCreation: disableForceTopicCreation, messageCh: messageCh, closeCh: make(chan struct{}), errorCh: make(chan error), dlq: dlq, log: log.WithField("topic", topic), } if options.Name != "" { consumer.consumerName = options.Name } else { consumer.consumerName = generateRandomName() } err := consumer.internalTopicSubscribeToPartitions() if err != nil { return nil, err } // set up timer to monitor for new partitions being added duration := options.AutoDiscoveryPeriod if duration <= 0 { duration = defaultAutoDiscoveryDuration } consumer.ticker = time.NewTicker(duration) go func() { for range consumer.ticker.C { consumer.log.Debug("Auto discovering new partitions") consumer.internalTopicSubscribeToPartitions() } }() return consumer, nil } func (c *consumer) internalTopicSubscribeToPartitions() error { partitions, err := c.client.TopicPartitions(c.topic) if err != nil { return err } oldNumPartitions := 0 newNumPartitions := len(partitions) c.Lock() defer c.Unlock() oldConsumers := c.consumers if oldConsumers != nil { oldNumPartitions = len(oldConsumers) if oldNumPartitions == newNumPartitions { c.log.Debug("Number of partitions in topic has not changed") return nil } c.log.WithField("old_partitions", oldNumPartitions). WithField("new_partitions", newNumPartitions). Info("Changed number of partitions in topic") } c.consumers = make([]*partitionConsumer, newNumPartitions) // Copy over the existing consumer instances for i := 0; i < oldNumPartitions; i++ { c.consumers[i] = oldConsumers[i] } type ConsumerError struct { err error partition int consumer *partitionConsumer } receiverQueueSize := c.options.ReceiverQueueSize metadata := c.options.Properties partitionsToAdd := newNumPartitions - oldNumPartitions var wg sync.WaitGroup ch := make(chan ConsumerError, partitionsToAdd) wg.Add(partitionsToAdd) for partitionIdx := oldNumPartitions; partitionIdx < newNumPartitions; partitionIdx++ { partitionTopic := partitions[partitionIdx] go func(idx int, pt string) { defer wg.Done() var nackRedeliveryDelay time.Duration if c.options.NackRedeliveryDelay == 0 { nackRedeliveryDelay = defaultNackRedeliveryDelay } else { nackRedeliveryDelay = c.options.NackRedeliveryDelay } opts := &partitionConsumerOpts{ topic: pt, consumerName: c.consumerName, subscription: c.options.SubscriptionName, subscriptionType: c.options.Type, subscriptionInitPos: c.options.SubscriptionInitialPosition, partitionIdx: idx, receiverQueueSize: receiverQueueSize, nackRedeliveryDelay: nackRedeliveryDelay, metadata: metadata, replicateSubscriptionState: c.options.ReplicateSubscriptionState, startMessageID: nil, subscriptionMode: durable, readCompacted: c.options.ReadCompacted, } cons, err := newPartitionConsumer(c, c.client, opts, c.messageCh, c.dlq) ch <- ConsumerError{ err: err, partition: idx, consumer: cons, } }(partitionIdx, partitionTopic) } go func() { wg.Wait() close(ch) }() for ce := range ch { if ce.err != nil { err = ce.err } else { c.consumers[ce.partition] = ce.consumer } } if err != nil { // Since there were some failures, // cleanup all the partitions that succeeded in creating the consumer for _, c := range c.consumers { if c != nil { c.Close() } } return err } return nil } func topicSubscribe(client *client, options ConsumerOptions, topic string, messageCh chan ConsumerMessage, dlqRouter *dlqRouter) (Consumer, error) { return newInternalConsumer(client, options, topic, messageCh, dlqRouter, false) } func (c *consumer) Subscription() string { return c.options.SubscriptionName } func (c *consumer) Unsubscribe() error { c.Lock() defer c.Unlock() var errMsg string for _, consumer := range c.consumers { if err := consumer.Unsubscribe(); err != nil { errMsg += fmt.Sprintf("topic %s, subscription %s: %s", consumer.topic, c.Subscription(), err) } } if errMsg != "" { return fmt.Errorf(errMsg) } return nil } func (c *consumer) Receive(ctx context.Context) (message Message, err error) { for { select { case <-c.closeCh: return nil, ErrConsumerClosed case cm, ok := <-c.messageCh: if !ok { return nil, ErrConsumerClosed } return cm.Message, nil case <-ctx.Done(): return nil, ctx.Err() } } } // Messages func (c *consumer) Chan() <-chan ConsumerMessage { return c.messageCh } // Ack the consumption of a single message func (c *consumer) Ack(msg Message) { c.AckID(msg.ID()) } // Ack the consumption of a single message, identified by its MessageID func (c *consumer) AckID(msgID MessageID) { mid, ok := c.messageID(msgID) if !ok { return } if mid.consumer != nil { mid.Ack() return } c.consumers[mid.partitionIdx].AckID(mid) } func (c *consumer) Nack(msg Message) { c.NackID(msg.ID()) } func (c *consumer) NackID(msgID MessageID) { mid, ok := c.messageID(msgID) if !ok { return } if mid.consumer != nil { mid.Nack() return } c.consumers[mid.partitionIdx].NackID(mid) } func (c *consumer) Close() { c.closeOnce.Do(func() { c.Lock() defer c.Unlock() var wg sync.WaitGroup for i := range c.consumers { wg.Add(1) go func(pc *partitionConsumer) { defer wg.Done() pc.Close() }(c.consumers[i]) } wg.Wait() close(c.closeCh) c.ticker.Stop() c.client.handlers.Del(c) c.dlq.close() }) } func (c *consumer) Seek(msgID MessageID) error { c.Lock() defer c.Unlock() if len(c.consumers) > 1 { return errors.New("for partition topic, seek command should perform on the individual partitions") } mid, ok := c.messageID(msgID) if !ok { return nil } return c.consumers[mid.partitionIdx].Seek(mid) } func (c *consumer) SeekByTime(time time.Time) error { c.Lock() defer c.Unlock() if len(c.consumers) > 1 { return errors.New("for partition topic, seek command should perform on the individual partitions") } return c.consumers[0].SeekByTime(time) } var r = &random{ R: rand.New(rand.NewSource(time.Now().UnixNano())), } type random struct { sync.Mutex R *rand.Rand } func generateRandomName() string { r.Lock() defer r.Unlock() chars := "abcdefghijklmnopqrstuvwxyz" bytes := make([]byte, 5) for i := range bytes { bytes[i] = chars[r.R.Intn(len(chars))] } return string(bytes) } func toProtoSubType(st SubscriptionType) pb.CommandSubscribe_SubType { switch st { case Exclusive: return pb.CommandSubscribe_Exclusive case Shared: return pb.CommandSubscribe_Shared case Failover: return pb.CommandSubscribe_Failover case KeyShared: return pb.CommandSubscribe_Key_Shared } return pb.CommandSubscribe_Exclusive } func toProtoInitialPosition(p SubscriptionInitialPosition) pb.CommandSubscribe_InitialPosition { switch p { case SubscriptionPositionLatest: return pb.CommandSubscribe_Latest case SubscriptionPositionEarliest: return pb.CommandSubscribe_Earliest } return pb.CommandSubscribe_Latest } func (c *consumer) messageID(msgID MessageID) (*messageID, bool) { mid, ok := msgID.(*messageID) if !ok { c.log.Warnf("invalid message id type") return nil, false } partition := mid.partitionIdx // did we receive a valid partition index? 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 nil, false } return mid, true }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "errors" "fmt" "sync" "time" pkgerrors "github.com/pkg/errors" log "github.com/sirupsen/logrus" ) type multiTopicConsumer struct { options ConsumerOptions messageCh chan ConsumerMessage consumers map[string]Consumer dlq *dlqRouter closeOnce sync.Once closeCh chan struct{} log *log.Entry } func newMultiTopicConsumer(client *client, options ConsumerOptions, topics []string, messageCh chan ConsumerMessage, dlq *dlqRouter) (Consumer, error) { mtc := &multiTopicConsumer{ options: options, messageCh: messageCh, consumers: make(map[string]Consumer, len(topics)), closeCh: make(chan struct{}), dlq: dlq, log: &log.Entry{}, } var errs error for ce := range subscriber(client, topics, options, messageCh, dlq) { if ce.err != nil { errs = pkgerrors.Wrapf(ce.err, "unable to subscribe to topic=%s", ce.topic) } else { mtc.consumers[ce.topic] = ce.consumer } } if errs != nil { for _, c := range mtc.consumers { c.Close() } return nil, errs } return mtc, nil } func (c *multiTopicConsumer) Subscription() string { return c.options.SubscriptionName } func (c *multiTopicConsumer) Unsubscribe() error { var errs error for t, consumer := range c.consumers { if err := consumer.Unsubscribe(); err != nil { msg := fmt.Sprintf("unable to 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 { case <-c.closeCh: return nil, ErrConsumerClosed case cm, ok := <-c.messageCh: if !ok { return nil, ErrConsumerClosed } return cm.Message, nil case <-ctx.Done(): return nil, ctx.Err() } } } // Messages func (c *multiTopicConsumer) Chan() <-chan ConsumerMessage { return c.messageCh } // Ack the consumption of a single message func (c *multiTopicConsumer) Ack(msg Message) { c.AckID(msg.ID()) } // Ack the consumption of a single message, identified by its MessageID func (c *multiTopicConsumer) AckID(msgID MessageID) { mid, ok := msgID.(*messageID) if !ok { c.log.Warnf("invalid message id type") return } if mid.consumer == nil { c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID) return } mid.Ack() } func (c *multiTopicConsumer) Nack(msg Message) { c.NackID(msg.ID()) } func (c *multiTopicConsumer) NackID(msgID MessageID) { mid, ok := msgID.(*messageID) if !ok { c.log.Warnf("invalid message id type") return } if mid.consumer == nil { c.log.Warnf("unable to nack messageID=%+v can not determine topic", msgID) return } mid.Nack() } func (c *multiTopicConsumer) Close() { c.closeOnce.Do(func() { var wg sync.WaitGroup wg.Add(len(c.consumers)) for _, con := range c.consumers { go func(consumer Consumer) { defer wg.Done() consumer.Close() }(con) } wg.Wait() close(c.closeCh) c.dlq.close() }) } func (c *multiTopicConsumer) Seek(msgID MessageID) error { return errors.New("seek command not allowed for multi topic consumer") } func (c *multiTopicConsumer) SeekByTime(time time.Time) error { return errors.New("seek command not allowed for multi topic consumer") }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "math" "sync" "time" "github.com/golang/protobuf/proto" log "github.com/sirupsen/logrus" "github.com/apache/pulsar-client-go/pulsar/internal" "github.com/apache/pulsar-client-go/pulsar/internal/compression" "github.com/apache/pulsar-client-go/pulsar/internal/pb" ) var ( compressionProviders = map[pb.CompressionType]compression.Provider{ pb.CompressionType_NONE: compression.NoopProvider, pb.CompressionType_LZ4: compression.Lz4Provider, pb.CompressionType_ZLIB: compression.ZLibProvider, pb.CompressionType_ZSTD: compression.ZStdProvider, } ) type consumerState int const ( consumerInit consumerState = iota consumerReady consumerClosing consumerClosed ) 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 // Lightweight subscription mode that doesn't have a durable cursor associated nonDurable ) type partitionConsumerOpts struct { topic string consumerName string subscription string subscriptionType SubscriptionType subscriptionInitPos SubscriptionInitialPosition partitionIdx int receiverQueueSize int nackRedeliveryDelay time.Duration metadata map[string]string replicateSubscriptionState bool startMessageID *messageID startMessageIDInclusive bool subscriptionMode subscriptionMode readCompacted bool disableForceTopicCreation bool } type partitionConsumer struct { client *client // this is needed for sending ConsumerMessage on the messageCh parentConsumer Consumer state consumerState options *partitionConsumerOpts conn internal.Connection topic string name string consumerID uint64 partitionIdx int // shared channel messageCh chan ConsumerMessage // the number of message slots available availablePermits int32 // the size of the queue channel for buffering messages queueSize int32 queueCh chan []*message startMessageID *messageID lastDequeuedMsg *messageID eventsCh chan interface{} connectedCh chan struct{} closeCh chan struct{} clearQueueCh chan func(id *messageID) nackTracker *negativeAcksTracker dlq *dlqRouter log *log.Entry } func newPartitionConsumer(parent Consumer, client *client, options *partitionConsumerOpts, messageCh chan ConsumerMessage, dlq *dlqRouter) (*partitionConsumer, error) { pc := &partitionConsumer{ state: consumerInit, parentConsumer: parent, client: client, options: options, topic: options.topic, name: options.consumerName, consumerID: client.rpcClient.NewConsumerID(), partitionIdx: options.partitionIdx, eventsCh: make(chan interface{}, 3), queueSize: int32(options.receiverQueueSize), queueCh: make(chan []*message, options.receiverQueueSize), startMessageID: options.startMessageID, connectedCh: make(chan struct{}), messageCh: messageCh, closeCh: make(chan struct{}), clearQueueCh: make(chan func(id *messageID)), dlq: dlq, log: log.WithField("topic", options.topic), } pc.log = pc.log.WithField("name", pc.name).WithField("subscription", options.subscription) pc.nackTracker = newNegativeAcksTracker(pc, options.nackRedeliveryDelay) err := pc.grabConn() if err != nil { log.WithError(err).Errorf("Failed to create consumer") return nil, err } pc.log.Info("Created consumer") pc.state = consumerReady go pc.dispatcher() go pc.runEventsLoop() return pc, nil } func (pc *partitionConsumer) Unsubscribe() error { req := &unsubscribeRequest{doneCh: make(chan struct{})} pc.eventsCh <- req // wait for the request to complete <-req.doneCh return req.err } func (pc *partitionConsumer) internalUnsubscribe(unsub *unsubscribeRequest) { defer close(unsub.doneCh) if pc.state == consumerClosed || pc.state == consumerClosing { pc.log.Error("Failed to unsubscribe consumer, the consumer is closing or consumer has been closed") return } pc.state = consumerClosing requestID := pc.client.rpcClient.NewRequestID() cmdUnsubscribe := &pb.CommandUnsubscribe{ RequestId: proto.Uint64(requestID), ConsumerId: proto.Uint64(pc.consumerID), } _, err := pc.client.rpcClient.RequestOnCnx(pc.conn, requestID, pb.BaseCommand_UNSUBSCRIBE, cmdUnsubscribe) if err != nil { pc.log.WithError(err).Error("Failed to unsubscribe consumer") unsub.err = err } pc.conn.DeleteConsumeHandler(pc.consumerID) if pc.nackTracker != nil { pc.nackTracker.Close() } pc.log.Infof("The consumer[%d] successfully unsubscribed", pc.consumerID) pc.state = consumerClosed } func (pc *partitionConsumer) getLastMessageID() (*messageID, error) { req := &getLastMsgIDRequest{doneCh: make(chan struct{})} pc.eventsCh <- req // wait for the request to complete <-req.doneCh return req.msgID, req.err } func (pc *partitionConsumer) internalGetLastMessageID(req *getLastMsgIDRequest) { defer close(req.doneCh) requestID := pc.client.rpcClient.NewRequestID() cmdGetLastMessageID := &pb.CommandGetLastMessageId{ RequestId: proto.Uint64(requestID), ConsumerId: proto.Uint64(pc.consumerID), } res, err := pc.client.rpcClient.RequestOnCnx(pc.conn, requestID, pb.BaseCommand_GET_LAST_MESSAGE_ID, cmdGetLastMessageID) if err != nil { pc.log.WithError(err).Error("Failed to get last message id") req.err = err } else { id := res.Response.GetLastMessageIdResponse.GetLastMessageId() req.msgID = convertToMessageID(id) } } func (pc *partitionConsumer) AckID(msgID *messageID) { if msgID != nil && msgID.ack() { req := &ackRequest{ msgID: msgID, } pc.eventsCh <- req } } func (pc *partitionConsumer) NackID(msgID *messageID) { pc.nackTracker.Add(msgID) } func (pc *partitionConsumer) Redeliver(msgIds []messageID) { pc.eventsCh <- &redeliveryRequest{msgIds} } func (pc *partitionConsumer) internalRedeliver(req *redeliveryRequest) { msgIds := req.msgIds pc.log.Debug("Request redelivery after negative ack for messages", msgIds) msgIDDataList := make([]*pb.MessageIdData, len(msgIds)) for i := 0; i < len(msgIds); i++ { msgIDDataList[i] = &pb.MessageIdData{ LedgerId: proto.Uint64(uint64(msgIds[i].ledgerID)), EntryId: proto.Uint64(uint64(msgIds[i].entryID)), } } pc.client.rpcClient.RequestOnCnxNoWait(pc.conn, pb.BaseCommand_REDELIVER_UNACKNOWLEDGED_MESSAGES, &pb.CommandRedeliverUnacknowledgedMessages{ ConsumerId: proto.Uint64(pc.consumerID), MessageIds: msgIDDataList, }) } func (pc *partitionConsumer) Close() { if pc.state != consumerReady { return } req := &closeRequest{doneCh: make(chan struct{})} pc.eventsCh <- req // wait for request to finish <-req.doneCh } func (pc *partitionConsumer) Seek(msgID *messageID) error { req := &seekRequest{ doneCh: make(chan struct{}), msgID: msgID, } pc.eventsCh <- req // wait for the request to complete <-req.doneCh return req.err } func (pc *partitionConsumer) internalSeek(seek *seekRequest) { defer close(seek.doneCh) if pc.state == consumerClosing || pc.state == consumerClosed { pc.log.Error("Consumer was already closed") return } id := &pb.MessageIdData{} err := proto.Unmarshal(seek.msgID.Serialize(), id) if err != nil { pc.log.WithError(err).Errorf("deserialize message id error: %s", err.Error()) seek.err = err } requestID := pc.client.rpcClient.NewRequestID() cmdSeek := &pb.CommandSeek{ ConsumerId: proto.Uint64(pc.consumerID), RequestId: proto.Uint64(requestID), MessageId: id, } _, err = pc.client.rpcClient.RequestOnCnx(pc.conn, requestID, pb.BaseCommand_SEEK, cmdSeek) if err != nil { pc.log.WithError(err).Error("Failed to reset to message id") seek.err = err } } func (pc *partitionConsumer) SeekByTime(time time.Time) error { req := &seekByTimeRequest{ doneCh: make(chan struct{}), publishTime: time, } pc.eventsCh <- req // wait for the request to complete <-req.doneCh return req.err } func (pc *partitionConsumer) internalSeekByTime(seek *seekByTimeRequest) { defer close(seek.doneCh) if pc.state == consumerClosing || pc.state == consumerClosed { pc.log.Error("Consumer was already closed") return } requestID := pc.client.rpcClient.NewRequestID() cmdSeek := &pb.CommandSeek{ ConsumerId: proto.Uint64(pc.consumerID), RequestId: proto.Uint64(requestID), MessagePublishTime: proto.Uint64(uint64(seek.publishTime.UnixNano() / int64(time.Millisecond))), } _, err := pc.client.rpcClient.RequestOnCnx(pc.conn, requestID, pb.BaseCommand_SEEK, cmdSeek) if err != nil { pc.log.WithError(err).Error("Failed to reset to message publish time") seek.err = err } } func (pc *partitionConsumer) internalAck(req *ackRequest) { msgID := req.msgID messageIDs := make([]*pb.MessageIdData, 1) messageIDs[0] = &pb.MessageIdData{ LedgerId: proto.Uint64(uint64(msgID.ledgerID)), EntryId: proto.Uint64(uint64(msgID.entryID)), } cmdAck := &pb.CommandAck{ ConsumerId: proto.Uint64(pc.consumerID), MessageId: messageIDs, AckType: pb.CommandAck_Individual.Enum(), } pc.client.rpcClient.RequestOnCnxNoWait(pc.conn, pb.BaseCommand_ACK, cmdAck) } func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, headersAndPayload internal.Buffer) error { pbMsgID := response.GetMessageId() reader := internal.NewMessageReader(headersAndPayload) msgMeta, err := reader.ReadMessageMetadata() if err != nil { pc.discardCorruptedMessage(pbMsgID, pb.CommandAck_ChecksumMismatch) return err } uncompressedHeadersAndPayload, err := pc.Decompress(msgMeta, headersAndPayload) if err != nil { pc.discardCorruptedMessage(pbMsgID, pb.CommandAck_DecompressionError) return err } // Reset the reader on the uncompressed buffer reader.ResetBuffer(uncompressedHeadersAndPayload) numMsgs := 1 if msgMeta.NumMessagesInBatch != nil { numMsgs = int(msgMeta.GetNumMessagesInBatch()) } messages := make([]*message, 0) var ackTracker *ackTracker // are there multiple messages in this batch? if numMsgs > 1 { ackTracker = newAckTracker(numMsgs) } for i := 0; i < numMsgs; i++ { smm, payload, err := reader.ReadMessage() if err != nil { pc.discardCorruptedMessage(pbMsgID, pb.CommandAck_BatchDeSerializeError) return err } msgID := newTrackingMessageID( int64(pbMsgID.GetLedgerId()), int64(pbMsgID.GetEntryId()), i, pc.partitionIdx, ackTracker) if pc.messageShouldBeDiscarded(msgID) { pc.AckID(msgID) continue } // set the consumer so we know how to ack the message id msgID.consumer = pc var msg *message if smm != nil { msg = &message{ publishTime: timeFromUnixTimestampMillis(msgMeta.GetPublishTime()), eventTime: timeFromUnixTimestampMillis(smm.GetEventTime()), key: smm.GetPartitionKey(), properties: internal.ConvertToStringMap(smm.GetProperties()), topic: pc.topic, msgID: msgID, payLoad: payload, replicationClusters: msgMeta.GetReplicateTo(), replicatedFrom: msgMeta.GetReplicatedFrom(), redeliveryCount: response.GetRedeliveryCount(), } } else { msg = &message{ publishTime: timeFromUnixTimestampMillis(msgMeta.GetPublishTime()), eventTime: timeFromUnixTimestampMillis(msgMeta.GetEventTime()), key: msgMeta.GetPartitionKey(), properties: internal.ConvertToStringMap(msgMeta.GetProperties()), topic: pc.topic, msgID: msgID, payLoad: payload, replicationClusters: msgMeta.GetReplicateTo(), replicatedFrom: msgMeta.GetReplicatedFrom(), redeliveryCount: response.GetRedeliveryCount(), } } messages = append(messages, msg) } // send messages to the dispatcher pc.queueCh <- messages return nil } func (pc *partitionConsumer) messageShouldBeDiscarded(msgID *messageID) bool { if pc.startMessageID == nil { return false } if pc.options.startMessageIDInclusive { return pc.startMessageID.greater(msgID) } // Non inclusive return pc.startMessageID.greaterEqual(msgID) } func (pc *partitionConsumer) ConnectionClosed() { // Trigger reconnection in the consumer goroutine pc.eventsCh <- &connectionClosed{} } // Flow command gives additional permits to send messages to the consumer. // A typical consumer implementation will use a queue to accumulate these messages // before the application is ready to consume them. After the consumer is ready, // the client needs to give permission to the broker to push messages. func (pc *partitionConsumer) internalFlow(permits uint32) error { if permits == 0 { return fmt.Errorf("invalid number of permits requested: %d", permits) } cmdFlow := &pb.CommandFlow{ ConsumerId: proto.Uint64(pc.consumerID), MessagePermits: proto.Uint32(permits), } pc.client.rpcClient.RequestOnCnxNoWait(pc.conn, pb.BaseCommand_FLOW, cmdFlow) return nil } // dispatcher manages the internal message queue channel // and manages the flow control func (pc *partitionConsumer) dispatcher() { defer func() { pc.log.Debug("exiting dispatch loop") }() var messages []*message for { var queueCh chan []*message var messageCh chan ConsumerMessage var nextMessage ConsumerMessage // are there more messages to send? if len(messages) > 0 { nextMessage = ConsumerMessage{ Consumer: pc.parentConsumer, Message: messages[0], } if pc.dlq.shouldSendToDlq(&nextMessage) { // pass the message to the DLQ router messageCh = pc.dlq.Chan() } else { // pass the message to application channel messageCh = pc.messageCh } } else { // we are ready for more messages queueCh = pc.queueCh } select { case <-pc.closeCh: return case _, ok := <-pc.connectedCh: if !ok { return } pc.log.Debug("dispatcher received connection event") messages = nil // reset available permits pc.availablePermits = 0 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") } case msgs, ok := <-queueCh: if !ok { return } // we only read messages here after the consumer has processed all messages // in the previous batch messages = msgs // if the messageCh is nil or the messageCh is full this will not be selected case messageCh <- nextMessage: // allow this message to be garbage collected messages[0] = nil messages = messages[1:] // TODO implement a better flow controller // send more permits if needed pc.availablePermits++ flowThreshold := int32(math.Max(float64(pc.queueSize/2), 1)) if pc.availablePermits >= flowThreshold { availablePermits := pc.availablePermits requestedPermits := availablePermits pc.availablePermits = 0 pc.log.Debugf("requesting more permits=%d available=%d", requestedPermits, availablePermits) if err := pc.internalFlow(uint32(requestedPermits)); err != nil { pc.log.WithError(err).Error("unable to send permits") } } 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 *messageID go func() { pc.queueCh <- nil }() for m := range pc.queueCh { // the queue has been drained if m == nil { break } else if nextMessageInQueue == nil { nextMessageInQueue = m[0].msgID.(*messageID) } } clearQueueCb(nextMessageInQueue) } } } type ackRequest struct { msgID *messageID } type unsubscribeRequest struct { doneCh chan struct{} err error } type closeRequest struct { doneCh chan struct{} } type redeliveryRequest struct { msgIds []messageID } type getLastMsgIDRequest struct { doneCh chan struct{} msgID *messageID err error } type seekRequest struct { doneCh chan struct{} msgID *messageID err error } type seekByTimeRequest struct { doneCh chan struct{} publishTime time.Time err error } func (pc *partitionConsumer) runEventsLoop() { defer func() { pc.log.Debug("exiting events loop") }() for { select { case <-pc.closeCh: return case i := <-pc.eventsCh: switch v := i.(type) { case *ackRequest: pc.internalAck(v) case *redeliveryRequest: pc.internalRedeliver(v) case *unsubscribeRequest: pc.internalUnsubscribe(v) case *getLastMsgIDRequest: pc.internalGetLastMessageID(v) case *seekRequest: pc.internalSeek(v) case *seekByTimeRequest: pc.internalSeekByTime(v) case *connectionClosed: pc.reconnectToBroker() case *closeRequest: pc.internalClose(v) return } } } } func (pc *partitionConsumer) internalClose(req *closeRequest) { defer close(req.doneCh) if pc.state != consumerReady { return } if pc.state == consumerClosed || pc.state == consumerClosing { pc.log.Error("The consumer is closing or has been closed") if pc.nackTracker != nil { pc.nackTracker.Close() } return } pc.state = consumerClosing pc.log.Infof("Closing consumer=%d", pc.consumerID) requestID := pc.client.rpcClient.NewRequestID() cmdClose := &pb.CommandCloseConsumer{ ConsumerId: proto.Uint64(pc.consumerID), RequestId: proto.Uint64(requestID), } _, err := pc.client.rpcClient.RequestOnCnx(pc.conn, requestID, pb.BaseCommand_CLOSE_CONSUMER, cmdClose) if err != nil { pc.log.WithError(err).Warn("Failed to close consumer") } else { pc.log.Info("Closed consumer") } pc.state = consumerClosed pc.conn.DeleteConsumeHandler(pc.consumerID) if pc.nackTracker != nil { pc.nackTracker.Close() } close(pc.closeCh) } func (pc *partitionConsumer) reconnectToBroker() { backoff := internal.Backoff{} for { if pc.state != consumerReady { // Consumer is already closing return } d := backoff.Next() pc.log.Info("Reconnecting to broker in ", d) time.Sleep(d) err := pc.grabConn() if err == nil { // Successfully reconnected pc.log.Info("Reconnected consumer to broker") return } } } func (pc *partitionConsumer) grabConn() error { lr, err := pc.client.lookupService.Lookup(pc.topic) 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) requestID := pc.client.rpcClient.NewRequestID() cmdSubscribe := &pb.CommandSubscribe{ Topic: proto.String(pc.topic), Subscription: proto.String(pc.options.subscription), SubType: subType.Enum(), ConsumerId: proto.Uint64(pc.consumerID), RequestId: proto.Uint64(requestID), ConsumerName: proto.String(pc.name), PriorityLevel: nil, Durable: proto.Bool(pc.options.subscriptionMode == durable), Metadata: internal.ConvertFromStringMap(pc.options.metadata), ReadCompacted: proto.Bool(pc.options.readCompacted), Schema: nil, InitialPosition: initialPosition.Enum(), ReplicateSubscriptionState: proto.Bool(pc.options.replicateSubscriptionState), } pc.startMessageID = pc.clearReceiverQueue() if pc.options.subscriptionMode != durable { // For regular subscriptions the broker will determine the restarting point cmdSubscribe.StartMessageId = convertToMessageIDData(pc.startMessageID) } if len(pc.options.metadata) > 0 { cmdSubscribe.Metadata = toKeyValues(pc.options.metadata) } // force topic creation is enabled by default so // we only need to set the flag when disabling it if pc.options.disableForceTopicCreation { cmdSubscribe.ForceTopicCreation = proto.Bool(false) } res, err := pc.client.rpcClient.Request(lr.LogicalAddr, lr.PhysicalAddr, requestID, pb.BaseCommand_SUBSCRIBE, cmdSubscribe) if err != nil { pc.log.WithError(err).Error("Failed to create consumer") return err } if res.Response.ConsumerStatsResponse != nil { pc.name = res.Response.ConsumerStatsResponse.GetConsumerName() } pc.conn = res.Cnx pc.log.Info("Connected consumer") pc.conn.AddConsumeHandler(pc.consumerID, pc) msgType := res.Response.GetType() switch msgType { case pb.BaseCommand_SUCCESS: // notify the dispatcher we have connection go func() { pc.connectedCh <- struct{}{} }() return nil case pb.BaseCommand_ERROR: errMsg := res.Response.GetError() return fmt.Errorf("%s: %s", errMsg.GetError().String(), errMsg.GetMessage()) default: return newUnexpectedErrMsg(msgType, requestID) } } func (pc *partitionConsumer) clearQueueAndGetNextMessage() *messageID { if pc.state != consumerReady { return nil } wg := &sync.WaitGroup{} wg.Add(1) var msgID *messageID pc.clearQueueCh <- func(id *messageID) { msgID = id wg.Done() } wg.Wait() return msgID } /** * 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() *messageID { nextMessageInQueue := pc.clearQueueAndGetNextMessage() if nextMessageInQueue != nil { return getPreviousMessage(nextMessageInQueue) } 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 } else { // No message was received or dequeued by this consumer. Next message would still be the startMessageId return pc.startMessageID } } func getPreviousMessage(mid *messageID) *messageID { if mid.batchIdx >= 0 { return &messageID{ ledgerID: mid.ledgerID, entryID: mid.entryID, batchIdx: mid.batchIdx - 1, partitionIdx: mid.partitionIdx, } } // Get on previous message in previous entry return &messageID{ ledgerID: mid.ledgerID, entryID: mid.entryID - 1, batchIdx: mid.batchIdx, partitionIdx: mid.partitionIdx, } } func (pc *partitionConsumer) Decompress(msgMeta *pb.MessageMetadata, payload internal.Buffer) (internal.Buffer, error) { provider, ok := compressionProviders[msgMeta.GetCompression()] if !ok { err := fmt.Errorf("unsupported compression type: %v", msgMeta.GetCompression()) pc.log.WithError(err).Error("Failed to decompress message.") return nil, err } uncompressed, err := provider.Decompress(payload.ReadableSlice(), int(msgMeta.GetUncompressedSize())) if err != nil { return nil, err } return internal.NewBufferWrapper(uncompressed), nil } func (pc *partitionConsumer) discardCorruptedMessage(msgID *pb.MessageIdData, validationError pb.CommandAck_ValidationError) { pc.log.WithFields(log.Fields{ "msgID": msgID, "validationError": validationError, }).Error("Discarding corrupted message") pc.client.rpcClient.RequestOnCnxNoWait(pc.conn, pb.BaseCommand_ACK, &pb.CommandAck{ ConsumerId: proto.Uint64(pc.consumerID), MessageId: []*pb.MessageIdData{msgID}, AckType: pb.CommandAck_Individual.Enum(), ValidationError: validationError.Enum(), }) } func convertToMessageIDData(msgID *messageID) *pb.MessageIdData { if msgID == nil { return nil } return &pb.MessageIdData{ LedgerId: proto.Uint64(uint64(msgID.ledgerID)), EntryId: proto.Uint64(uint64(msgID.entryID)), } } func convertToMessageID(id *pb.MessageIdData) *messageID { if id == nil { return nil } msgID := &messageID{ ledgerID: int64(*id.LedgerId), entryID: int64(*id.EntryId), } if id.BatchIndex != nil { msgID.batchIdx = int(*id.BatchIndex) } return msgID }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "errors" "fmt" "regexp" "strings" "sync" "time" pkgerrors "github.com/pkg/errors" log "github.com/sirupsen/logrus" "github.com/apache/pulsar-client-go/pulsar/internal" ) const ( defaultAutoDiscoveryDuration = 1 * time.Minute ) type regexConsumer struct { client *client dlq *dlqRouter options ConsumerOptions messageCh chan ConsumerMessage namespace string pattern *regexp.Regexp consumersLock sync.Mutex consumers map[string]Consumer subscribeCh chan []string unsubscribeCh chan []string closeOnce sync.Once closeCh chan struct{} ticker *time.Ticker log *log.Entry } func newRegexConsumer(c *client, opts ConsumerOptions, tn *internal.TopicName, pattern *regexp.Regexp, msgCh chan ConsumerMessage, dlq *dlqRouter) (Consumer, error) { rc := ®exConsumer{ client: c, dlq: dlq, options: opts, messageCh: msgCh, namespace: tn.Namespace, pattern: pattern, consumers: make(map[string]Consumer), subscribeCh: make(chan []string, 1), unsubscribeCh: make(chan []string, 1), closeCh: make(chan struct{}), log: log.WithField("topic", tn.Name), } topics, err := rc.topics() if err != nil { return nil, err } var errs error for ce := range subscriber(c, topics, opts, msgCh, dlq) { if ce.err != nil { errs = pkgerrors.Wrapf(ce.err, "unable to subscribe to topic=%s", ce.topic) } else { rc.consumers[ce.topic] = ce.consumer } } if errs != nil { for _, c := range rc.consumers { c.Close() } return nil, errs } // set up timer duration := opts.AutoDiscoveryPeriod if duration <= 0 { duration = defaultAutoDiscoveryDuration } rc.ticker = time.NewTicker(duration) go rc.monitor() return rc, nil } func (c *regexConsumer) Subscription() string { return c.options.SubscriptionName } func (c *regexConsumer) Unsubscribe() error { var errs error c.consumersLock.Lock() defer c.consumersLock.Unlock() for topic, consumer := range c.consumers { if err := consumer.Unsubscribe(); err != nil { msg := fmt.Sprintf("unable to 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 { case <-c.closeCh: return nil, ErrConsumerClosed case cm, ok := <-c.messageCh: if !ok { return nil, ErrConsumerClosed } return cm.Message, nil case <-ctx.Done(): return nil, ctx.Err() } } } // Chan func (c *regexConsumer) Chan() <-chan ConsumerMessage { return c.messageCh } // Ack the consumption of a single message func (c *regexConsumer) Ack(msg Message) { c.AckID(msg.ID()) } // Ack the consumption of a single message, identified by its MessageID func (c *regexConsumer) AckID(msgID MessageID) { mid, ok := msgID.(*messageID) if !ok { c.log.Warnf("invalid message id type") return } if mid.consumer == nil { c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID) return } mid.Ack() } func (c *regexConsumer) Nack(msg Message) { c.NackID(msg.ID()) } func (c *regexConsumer) NackID(msgID MessageID) { mid, ok := msgID.(*messageID) if !ok { c.log.Warnf("invalid message id type") return } if mid.consumer == nil { c.log.Warnf("unable to nack messageID=%+v can not determine topic", msgID) return } mid.Nack() } func (c *regexConsumer) Close() { c.closeOnce.Do(func() { c.ticker.Stop() close(c.closeCh) var wg sync.WaitGroup c.consumersLock.Lock() defer c.consumersLock.Unlock() wg.Add(len(c.consumers)) for _, con := range c.consumers { go func(consumer Consumer) { defer wg.Done() consumer.Close() }(con) } wg.Wait() c.dlq.close() }) } func (c *regexConsumer) Seek(msgID MessageID) error { return errors.New("seek command not allowed for regex consumer") } func (c *regexConsumer) SeekByTime(time time.Time) error { return errors.New("seek command not allowed for regex consumer") } func (c *regexConsumer) closed() bool { select { case <-c.closeCh: return true default: return false } } func (c *regexConsumer) monitor() { for { select { case <-c.closeCh: return case <-c.ticker.C: c.log.Debug("Auto discovering topics") if !c.closed() { c.discover() } case topics := <-c.subscribeCh: if len(topics) > 0 && !c.closed() { c.subscribe(topics, c.dlq) } case topics := <-c.unsubscribeCh: if len(topics) > 0 && !c.closed() { c.unsubscribe(topics) } } } } func (c *regexConsumer) discover() { topics, err := c.topics() if err != nil { c.log.WithError(err).Errorf("Failed to discover topics") return } known := c.knownTopics() newTopics := topicsDiff(topics, known) staleTopics := topicsDiff(known, topics) if log.GetLevel() == log.DebugLevel { l := c.log.WithFields(log.Fields{ "new_topics": newTopics, "old_topics": staleTopics, }) l.Debug("discover topics") } c.unsubscribeCh <- staleTopics c.subscribeCh <- newTopics } func (c *regexConsumer) knownTopics() []string { c.consumersLock.Lock() defer c.consumersLock.Unlock() topics := make([]string, len(c.consumers)) n := 0 for t := range c.consumers { topics[n] = t n++ } return topics } func (c *regexConsumer) subscribe(topics []string, dlq *dlqRouter) { if log.GetLevel() == log.DebugLevel { c.log.WithField("topics", topics).Debug("subscribe") } consumers := make(map[string]Consumer, len(topics)) for ce := range subscriber(c.client, topics, c.options, c.messageCh, dlq) { if ce.err != nil { c.log.Warnf("Failed to subscribe to topic=%s", ce.topic) } else { consumers[ce.topic] = ce.consumer } } c.consumersLock.Lock() defer c.consumersLock.Unlock() for t, consumer := range consumers { c.consumers[t] = consumer } } func (c *regexConsumer) unsubscribe(topics []string) { if log.GetLevel() == log.DebugLevel { c.log.WithField("topics", topics).Debug("unsubscribe") } consumers := make(map[string]Consumer, len(topics)) c.consumersLock.Lock() for _, t := range topics { if consumer, ok := c.consumers[t]; ok { consumers[t] = consumer delete(c.consumers, t) } } c.consumersLock.Unlock() for t, consumer := range consumers { log.Debugf("unsubscribe from topic=%s subscription=%s", t, c.options.SubscriptionName) if err := consumer.Unsubscribe(); err != nil { c.log.Warnf("unable to unsubscribe from topic=%s subscription=%s", t, c.options.SubscriptionName) } consumer.Close() } } func (c *regexConsumer) topics() ([]string, error) { topics, err := c.client.namespaceTopics(c.namespace) if err != nil { return nil, err } filtered := filterTopics(topics, c.pattern) return filtered, nil } type consumerError struct { err error topic string consumer Consumer } func subscriber(c *client, topics []string, opts ConsumerOptions, ch chan ConsumerMessage, dlq *dlqRouter) <-chan consumerError { consumerErrorCh := make(chan consumerError, len(topics)) var wg sync.WaitGroup wg.Add(len(topics)) go func() { wg.Wait() close(consumerErrorCh) }() for _, t := range topics { go func(topic string) { defer wg.Done() c, err := newInternalConsumer(c, opts, topic, ch, dlq, true) consumerErrorCh <- consumerError{ err: err, topic: topic, consumer: c, } }(t) } return consumerErrorCh } func filterTopics(topics []string, regex *regexp.Regexp) []string { matches := make(map[string]bool) matching := make([]string, 0) for _, t := range topics { tn, _ := internal.ParseTopicName(t) topic := internal.TopicNameWithoutPartitionPart(tn) if _, ok := matches[topic]; ok { continue } if regex.MatchString(topic) { matches[topic] = true matching = append(matching, topic) } } return matching } // topicDiff returns all topics in topics1 that are not in topics2 func topicsDiff(topics1 []string, topics2 []string) []string { if len(topics2) == 0 { return topics1 } diff := make([]string, 0) topics := make(map[string]bool, len(topics2)) for _, t := range topics2 { topics[t] = true } for _, t := range topics1 { if !topics[t] { diff = append(diff, t) } } return diff } func extractTopicPattern(tn *internal.TopicName) (*regexp.Regexp, error) { idx := strings.Index(tn.Name, tn.Namespace) if idx > 0 { return regexp.Compile(tn.Name[idx:]) } return regexp.Compile(tn.Name) }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "errors" "time" "github.com/apache/pulsar-client-go/pulsar/internal" log "github.com/sirupsen/logrus" ) type dlqRouter struct { client Client producer Producer policy *DLQPolicy messageCh chan ConsumerMessage closeCh chan interface{} log *log.Entry } func newDlqRouter(client Client, policy *DLQPolicy) (*dlqRouter, error) { r := &dlqRouter{ client: client, policy: policy, } if policy != nil { if policy.MaxDeliveries <= 0 { return nil, errors.New("DLQPolicy.MaxDeliveries needs to be > 0") } if policy.Topic == "" { return nil, errors.New("DLQPolicy.Topic needs to be set to a valid topic name") } r.messageCh = make(chan ConsumerMessage) r.closeCh = make(chan interface{}, 1) r.log = log.WithField("dlq-topic", policy.Topic) go r.run() } return r, nil } func (r *dlqRouter) shouldSendToDlq(cm *ConsumerMessage) bool { if r.policy == nil { return false } msg := cm.Message.(*message) r.log.WithField("count", msg.redeliveryCount). WithField("max", r.policy.MaxDeliveries). WithField("msgId", msg.msgID). Debug("Should route to DLQ?") // We use >= here because we're comparing the number of re-deliveries with // the number of deliveries. So: // * the user specifies that wants to process a message up to 10 times. // * the first time, the redeliveryCount == 0, then 1 and so on // * when we receive the message and redeliveryCount == 10, it means // that the application has already got (and Nack()) the message 10 // times, so this time we should just go to DLQ. return msg.redeliveryCount >= r.policy.MaxDeliveries } func (r *dlqRouter) Chan() chan ConsumerMessage { return r.messageCh } func (r *dlqRouter) run() { for { select { case cm := <-r.messageCh: r.log.WithField("msgID", cm.ID()).Debug("Got message for DLQ") producer := r.getProducer() msg := cm.Message.(*message) msgID := msg.ID() producer.SendAsync(context.Background(), &ProducerMessage{ Payload: msg.Payload(), Key: msg.Key(), Properties: msg.Properties(), EventTime: msg.EventTime(), ReplicationClusters: msg.replicationClusters, }, func(MessageID, *ProducerMessage, error) { r.log.WithField("msgID", msgID).Debug("Sent message to DLQ") cm.Consumer.AckID(msgID) }) case <-r.closeCh: if r.producer != nil { r.producer.Close() } r.log.Debug("Closed DLQ router") return } } } func (r *dlqRouter) close() { // Attempt to write on the close channel, without blocking select { case r.closeCh <- nil: default: } } func (r *dlqRouter) getProducer() Producer { if r.producer != nil { // Producer was already initialized return r.producer } // Retry to create producer indefinitely backoff := &internal.Backoff{} for { producer, err := r.client.CreateProducer(ProducerOptions{ Topic: r.policy.Topic, CompressionType: LZ4, BatchingMaxPublishDelay: 100 * time.Millisecond, }) if err != nil { r.log.WithError(err).Error("Failed to create DLQ producer") time.Sleep(backoff.Next()) continue } else { r.producer = producer return producer } } }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" // Result used to represent pulsar processing is an alias of type int. type Result int const ( // ResultOk means no errors ResultOk Result = iota // ResultUnknownError means unknown error happened on broker ResultUnknownError // ResultInvalidConfiguration means invalid configuration ResultInvalidConfiguration // ResultTimeoutError means operation timed out ResultTimeoutError // ResultLookupError means broker lookup failed ResultLookupError // ResultInvalidTopicName means invalid topic name ResultInvalidTopicName // ResultConnectError means failed to connect to broker ResultConnectError // ReadError means failed to read from socket //ReadError Result = 6 // AuthenticationError means authentication failed on broker //AuthenticationError Result = 7 //AuthorizationError Result = 8 //ErrorGettingAuthenticationData Result = 9 // Client cannot find authorization data //BrokerMetadataError Result = 10 // Broker failed in updating metadata //BrokerPersistenceError Result = 11 // Broker failed to persist entry //ChecksumError Result = 12 // Corrupt message checksum failure // ConsumerBusy means Exclusive consumer is already connected ConsumerBusy Result = 13 //NotConnectedError Result = 14 // Producer/Consumer is not currently connected to broker //AlreadyClosedError Result = 15 // Producer/Consumer is already closed and not accepting any operation //InvalidMessage Result = 16 // Error in publishing an already used message //ConsumerNotInitialized Result = 17 // Consumer is not initialized //ProducerNotInitialized Result = 18 // Producer is not initialized //TooManyLookupRequestException Result = 19 // Too Many concurrent LookupRequest // InvalidUrl means Client Initialized with Invalid Broker Url (VIP Url passed to Client Constructor) //InvalidUrl Result = 21 // ServiceUnitNotReady unloaded between client did lookup and producer/consumer got created //ServiceUnitNotReady Result = 22 //OperationNotSupported Result = 23 //ProducerBlockedQuotaExceededError Result = 24 // Producer is blocked //ProducerBlockedQuotaExceededException Result = 25 // Producer is getting exception //ProducerQueueIsFull Result = 26 // Producer queue is full //MessageTooBig Result = 27 // Trying to send a messages exceeding the max size TopicNotFound Result = 28 // Topic not found SubscriptionNotFound Result = 29 // Subscription not found //ConsumerNotFound Result = 30 // Consumer not found // UnsupportedVersionError when an older client/version doesn't support a required feature //UnsupportedVersionError Result = 31 //TopicTerminated Result = 32 // Topic was already terminated //CryptoError Result = 33 // Error when crypto operation fails ) // Error implement error interface, composed of two parts: msg and result. type Error struct { msg string result Result } func (e *Error) Result() Result { return e.result } func (e *Error) Error() string { return e.msg } func newError(result Result, msg string) error { return &Error{ msg: fmt.Sprintf("%s: %s", msg, getResultStr(result)), result: result, } } func getResultStr(r Result) string { switch r { case ResultOk: return "OK" case ResultUnknownError: return "Unknown error" case ResultInvalidConfiguration: return "InvalidConfiguration" case ResultTimeoutError: return "TimeoutError" case ResultLookupError: return "LookupError" case ResultInvalidTopicName: return "InvalidTopicName" case ResultConnectError: return "ConnectError" case ConsumerBusy: return "ConsumerBusy" default: return fmt.Sprintf("Result(%d)", r) } }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" pkgerrors "github.com/pkg/errors" "github.com/golang/protobuf/proto" "github.com/apache/pulsar-client-go/pulsar/internal" "github.com/apache/pulsar-client-go/pulsar/internal/pb" ) // NewUnexpectedErrMsg instantiates an ErrUnexpectedMsg error. // Optionally provide a list of IDs associated with the message // for additional context in the error message. func newUnexpectedErrMsg(msgType pb.BaseCommand_Type, ids ...interface{}) *unexpectedErrMsg { return &unexpectedErrMsg{ msgType: msgType, ids: ids, } } // UnexpectedErrMsg is returned when an unexpected message is received. type unexpectedErrMsg struct { msgType pb.BaseCommand_Type ids []interface{} } // Error satisfies the error interface. func (e *unexpectedErrMsg) Error() string { msg := fmt.Sprintf("received unexpected message of type %q", e.msgType.String()) for _, id := range e.ids { msg += fmt.Sprintf(" consumerID=%v", id) } return msg } func validateTopicNames(topics ...string) error { var errs error for _, t := range topics { if _, err := internal.ParseTopicName(t); err != nil { errs = pkgerrors.Wrapf(err, "invalid topic name: %s", t) } } return errs } func toKeyValues(metadata map[string]string) []*pb.KeyValue { kvs := make([]*pb.KeyValue, 0, len(metadata)) for k, v := range metadata { kv := &pb.KeyValue{ Key: proto.String(k), Value: proto.String(v), } kvs = append(kvs, kv) } return kvs }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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/big" "strings" "sync" "time" "github.com/golang/protobuf/proto" "github.com/apache/pulsar-client-go/pulsar/internal/pb" ) type messageID struct { ledgerID int64 entryID int64 batchIdx int partitionIdx int tracker *ackTracker consumer acker } func (id *messageID) Ack() { if id.consumer == nil { return } if id.ack() { id.consumer.AckID(id) } } func (id *messageID) Nack() { if id.consumer == nil { return } id.consumer.NackID(id) } func (id *messageID) ack() bool { if id.tracker != nil && id.batchIdx > -1 { return id.tracker.ack(id.batchIdx) } return true } func (id *messageID) greater(other *messageID) bool { if id.ledgerID != other.ledgerID { return id.ledgerID > other.ledgerID } if id.entryID != other.entryID { return id.entryID > other.entryID } return id.batchIdx > other.batchIdx } 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 { return id.equal(other) || id.greater(other) } func (id *messageID) Serialize() []byte { msgID := &pb.MessageIdData{ LedgerId: proto.Uint64(uint64(id.ledgerID)), EntryId: proto.Uint64(uint64(id.entryID)), BatchIndex: proto.Int(id.batchIdx), Partition: proto.Int(id.partitionIdx), } data, _ := proto.Marshal(msgID) return data } func deserializeMessageID(data []byte) (MessageID, error) { msgID := &pb.MessageIdData{} err := proto.Unmarshal(data, msgID) if err != nil { return nil, err } id := newMessageID( int64(msgID.GetLedgerId()), int64(msgID.GetEntryId()), int(msgID.GetBatchIndex()), int(msgID.GetPartition()), ) return id, nil } func newMessageID(ledgerID int64, entryID int64, batchIdx int, partitionIdx int) MessageID { return &messageID{ ledgerID: ledgerID, entryID: entryID, batchIdx: batchIdx, partitionIdx: partitionIdx, } } func newTrackingMessageID(ledgerID int64, entryID int64, batchIdx int, partitionIdx int, tracker *ackTracker) *messageID { return &messageID{ ledgerID: ledgerID, entryID: entryID, batchIdx: batchIdx, partitionIdx: partitionIdx, tracker: tracker, } } func timeFromUnixTimestampMillis(timestamp uint64) time.Time { ts := int64(timestamp) * int64(time.Millisecond) seconds := ts / int64(time.Second) nanos := ts - (seconds * int64(time.Second)) return time.Unix(seconds, nanos) } type message struct { publishTime time.Time eventTime time.Time key string payLoad []byte msgID MessageID properties map[string]string topic string replicationClusters []string replicatedFrom string redeliveryCount uint32 } func (msg *message) Topic() string { return msg.topic } func (msg *message) Properties() map[string]string { return msg.properties } func (msg *message) Payload() []byte { return msg.payLoad } func (msg *message) ID() MessageID { return msg.msgID } func (msg *message) PublishTime() time.Time { return msg.publishTime } func (msg *message) EventTime() time.Time { return msg.eventTime } func (msg *message) Key() string { return msg.key } func (msg *message) RedeliveryCount() uint32 { return msg.redeliveryCount } func (msg *message) IsReplicated() bool { return msg.replicatedFrom != "" } func (msg *message) GetReplicatedFrom() string { return msg.replicatedFrom } 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) } return &ackTracker{ size: size, batchIDs: batchIDs, } } type ackTracker struct { sync.Mutex size int batchIDs *big.Int } func (t *ackTracker) ack(batchID int) bool { if batchID < 0 { return true } t.Lock() defer t.Unlock() t.batchIDs = t.batchIDs.SetBit(t.batchIDs, batchID, 0) return len(t.batchIDs.Bits()) == 0 } func (t *ackTracker) completed() bool { t.Lock() defer t.Unlock() return len(t.batchIDs.Bits()) == 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 auth import ( "crypto/tls" "encoding/base64" "errors" "io/ioutil" "regexp" "strings" "time" zms "github.com/yahoo/athenz/libs/go/zmssvctoken" zts "github.com/yahoo/athenz/libs/go/ztsroletoken" ) const ( minExpire = 2 * time.Hour maxExpire = 24 * time.Hour ) 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 } type privateKeyURI struct { Scheme string MediaTypeAndEncodingType string Data string Path string } func NewAuthenticationAthenzWithParams(params map[string]string) (Provider, error) { return NewAuthenticationAthenz( params["providerDomain"], params["tenantDomain"], params["tenantService"], params["privateKey"], params["keyId"], params["principalHeader"], params["ztsUrl"], ), nil } func NewAuthenticationAthenz( providerDomain string, tenantDomain string, tenantService string, privateKey string, keyID string, principalHeader string, ztsURL string) Provider { var fixedKeyID string if keyID == "" { fixedKeyID = "0" } else { fixedKeyID = keyID } ztsNewRoleToken := func(tok zms.Token, domain string, opts zts.RoleTokenOptions) zts.RoleToken { return zts.RoleToken(zts.NewRoleToken(tok, 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, } } func (p *athenzAuthProvider) Init() error { uriSt := parseURI(p.privateKey) var keyData []byte if uriSt.Scheme == "data" { if uriSt.MediaTypeAndEncodingType != "application/x-pem-file;base64" { return errors.New("Unsupported mediaType or encodingType: " + uriSt.MediaTypeAndEncodingType) } key, err := base64.StdEncoding.DecodeString(uriSt.Data) if err != nil { return err } keyData = key } else if uriSt.Scheme == "file" { key, err := ioutil.ReadFile(uriSt.Path) if err != nil { return err } keyData = key } else { return errors.New("Unsupported URI Scheme: " + uriSt.Scheme) } tb, err := p.zmsNewTokenBuilder(p.tenantDomain, p.tenantService, keyData, p.keyID) if err != nil { return err } 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 } func (p *athenzAuthProvider) Name() string { return "athenz" } func (p *athenzAuthProvider) GetTLSCertificate() (*tls.Certificate, error) { return nil, nil } func (p *athenzAuthProvider) GetData() ([]byte, error) { tok, err := p.roleToken.RoleTokenValue() if err != nil { return nil, err } return []byte(tok), nil } func (p *athenzAuthProvider) Close() error { return nil } func parseURI(uri string) privateKeyURI { var uriSt privateKeyURI // scheme mediatype[;base64] path file const expression = `^(?:([^:/?#]+):)(?:([;/\\\-\w]*),)?(?:/{0,2}((?:[^?#/]*/)*))?([^?#]*)` // when expression cannot be parsed, then panics re := regexp.MustCompile(expression) if re.MatchString(uri) { groups := re.FindStringSubmatch(uri) uriSt.Scheme = groups[1] uriSt.MediaTypeAndEncodingType = groups[2] uriSt.Data = groups[4] uriSt.Path = groups[3] + groups[4] } return uriSt }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 disabled struct{} // NewAuthDisabled return a interface of Provider func NewAuthDisabled() Provider { return &disabled{} } func (disabled) Init() error { return nil } func (disabled) GetData() ([]byte, error) { return nil, nil } func (disabled) Name() string { return "" } func (disabled) GetTLSCertificate() (*tls.Certificate, error) { return nil, nil } func (disabled) Close() error { return nil }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "encoding/json" "fmt" "io" "github.com/pkg/errors" ) // Provider is a interface of authentication providers. type Provider interface { // Init the authentication provider. Init() error // Name func returns the identifier for this authentication method. Name() string // return a client certificate chain, or nil if the data are not available GetTLSCertificate() (*tls.Certificate, error) // GetData returns the authentication data identifying this client that will be sent to the broker. GetData() ([]byte, error) io.Closer } // NewProvider get/create an authentication data provider which provides the data // that this client will be sent to the broker. // Some authentication method need to auth between each client channel. So it need // the broker, who it will talk to. func NewProvider(name string, params string) (Provider, error) { m := parseParams(params) switch name { case "": return NewAuthDisabled(), nil case "tls", "org.apache.pulsar.client.impl.auth.AuthenticationTls": return NewAuthenticationTLSWithParams(m), nil case "token", "org.apache.pulsar.client.impl.auth.AuthenticationToken": return NewAuthenticationTokenWithParams(m) case "athenz", "org.apache.pulsar.client.impl.auth.AuthenticationAthenz": return NewAuthenticationAthenzWithParams(m) default: return nil, errors.New(fmt.Sprintf("invalid auth provider '%s'", name)) } } func parseParams(params string) map[string]string { var mapString map[string]string json.Unmarshal([]byte(params), &mapString) return mapString }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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) Provider { return NewAuthenticationTLS( params["tlsCertFile"], params["tlsKeyFile"], ) } // NewAuthenticationTLS initialize the authentication provider func NewAuthenticationTLS(certificatePath string, privateKeyPath string) Provider { 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 } func (p *tlsAuthProvider) GetData() ([]byte, error) { return nil, nil } func (tlsAuthProvider) Close() error { return nil }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "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) (Provider, error) { if params["token"] != "" { return NewAuthenticationToken(params["token"]), nil } else if params["file"] != "" { return NewAuthenticationTokenFromFile(params["file"]), nil } else { return nil, errors.New("missing configuration for token auth") } } // NewAuthenticationToken returns a token auth provider that will use the specified token to // talk with Pulsar brokers func NewAuthenticationToken(token string) Provider { return &tokenAuthProvider{ tokenSupplier: func() (string, error) { if token == "" { return "", errors.New("empty token credentials") } return token, nil }, } } // NewAuthenticationTokenFromSupplier returns a token auth provider that get // the token data from a user supplied function. The function is invoked each // time the client library needs to use a token in talking with Pulsar brokers func NewAuthenticationTokenFromSupplier(tokenSupplier func() (string, error)) Provider { return &tokenAuthProvider{ tokenSupplier: tokenSupplier, } } // NewAuthenticationTokenFromFile return a interface of a Provider with a string token file path. func NewAuthenticationTokenFromFile(tokenFilePath string) Provider { 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) Name() string { return "token" } func (p *tokenAuthProvider) GetTLSCertificate() (*tls.Certificate, error) { return nil, nil } func (p *tokenAuthProvider) GetData() ([]byte, error) { t, err := p.tokenSupplier() if err != nil { return nil, err } return []byte(t), nil } func (tokenAuthProvider) Close() error { return nil }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" ) // Backoff type Backoff struct { backoff time.Duration } const ( minBackoff = 100 * time.Millisecond maxBackoff = 60 * time.Second ) // Next func (b *Backoff) Next() time.Duration { // Double the delay each time b.backoff += b.backoff if b.backoff.Nanoseconds() < minBackoff.Nanoseconds() { b.backoff = minBackoff } else if b.backoff.Nanoseconds() > maxBackoff.Nanoseconds() { b.backoff = maxBackoff } return b.backoff }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "fmt" "time" "github.com/apache/pulsar-client-go/pulsar/internal/compression" "github.com/apache/pulsar-client-go/pulsar/internal/pb" "github.com/golang/protobuf/proto" log "github.com/sirupsen/logrus" ) const ( // MaxMessageSize limit message size for transfer MaxMessageSize = 5 * 1024 * 1024 // MaxBatchSize will be the largest size for a batch sent from this particular producer. // This is used as a baseline to allocate a new buffer that can hold the entire batch // without needing costly re-allocations. MaxBatchSize = 128 * 1024 // DefaultMaxMessagesPerBatch init default num of entries in per batch. DefaultMaxMessagesPerBatch = 1000 ) // BatchBuilder wraps the objects needed to build a batch. type BatchBuilder struct { buffer Buffer // Current number of messages in the batch numMessages uint // Max number of message allowed in the batch maxMessages uint producerName string producerID uint64 cmdSend *pb.BaseCommand msgMetadata *pb.MessageMetadata callbacks []interface{} compressionProvider compression.Provider } // NewBatchBuilder init batch builder and return BatchBuilder pointer. Build a new batch message container. func NewBatchBuilder(maxMessages uint, producerName string, producerID uint64, compressionType pb.CompressionType) (*BatchBuilder, error) { if maxMessages == 0 { maxMessages = DefaultMaxMessagesPerBatch } bb := &BatchBuilder{ buffer: NewBuffer(4096), numMessages: 0, maxMessages: maxMessages, producerName: producerName, producerID: producerID, cmdSend: baseCommand(pb.BaseCommand_SEND, &pb.CommandSend{ ProducerId: &producerID, }), msgMetadata: &pb.MessageMetadata{ ProducerName: &producerName, }, callbacks: []interface{}{}, compressionProvider: getCompressionProvider(compressionType), } if compressionType != pb.CompressionType_NONE { bb.msgMetadata.Compression = &compressionType } if !bb.compressionProvider.CanCompress() { return nil, fmt.Errorf("compression provider %d can only decompress data", compressionType) } return bb, nil } // IsFull check if the size in the current batch exceeds the maximum size allowed by the batch func (bb *BatchBuilder) IsFull() bool { return bb.numMessages >= bb.maxMessages || bb.buffer.ReadableBytes() > MaxBatchSize } func (bb *BatchBuilder) hasSpace(payload []byte) bool { msgSize := uint32(len(payload)) return bb.numMessages > 0 && (bb.buffer.ReadableBytes()+msgSize) > MaxBatchSize } // Add will add single message to batch. func (bb *BatchBuilder) Add(metadata *pb.SingleMessageMetadata, sequenceID uint64, payload []byte, callback interface{}, replicateTo []string, deliverAt time.Time) bool { if replicateTo != nil && bb.numMessages != 0 { // If the current batch is not empty and we're trying to set the replication clusters, // then we need to force the current batch to flush and send the message individually return false } else if bb.msgMetadata.ReplicateTo != nil { // There's already a message with cluster replication list. need to flush before next // message can be sent return false } else if bb.hasSpace(payload) { // The current batch is full. Producer has to call Flush() to return false } if bb.numMessages == 0 { bb.msgMetadata.SequenceId = proto.Uint64(sequenceID) bb.msgMetadata.PublishTime = proto.Uint64(TimestampMillis(time.Now())) bb.msgMetadata.SequenceId = proto.Uint64(sequenceID) bb.msgMetadata.ProducerName = &bb.producerName bb.msgMetadata.ReplicateTo = replicateTo bb.msgMetadata.PartitionKey = metadata.PartitionKey if deliverAt.UnixNano() > 0 { bb.msgMetadata.DeliverAtTime = proto.Int64(int64(TimestampMillis(deliverAt))) } bb.cmdSend.Send.SequenceId = proto.Uint64(sequenceID) } addSingleMessageToBatch(bb.buffer, metadata, payload) bb.numMessages++ bb.callbacks = append(bb.callbacks, callback) return true } func (bb *BatchBuilder) reset() { bb.numMessages = 0 bb.buffer.Clear() bb.callbacks = []interface{}{} bb.msgMetadata.ReplicateTo = nil } // Flush all the messages buffered in the client and wait until all messages have been successfully persisted. func (bb *BatchBuilder) Flush() (batchData []byte, sequenceID uint64, callbacks []interface{}) { if bb.numMessages == 0 { // No-Op for empty batch return nil, 0, nil } log.Debug("BatchBuilder flush: messages: ", bb.numMessages) bb.msgMetadata.NumMessagesInBatch = proto.Int32(int32(bb.numMessages)) bb.cmdSend.Send.NumMessages = proto.Int32(int32(bb.numMessages)) uncompressedSize := bb.buffer.ReadableBytes() compressed := bb.compressionProvider.Compress(bb.buffer.ReadableSlice()) bb.msgMetadata.UncompressedSize = &uncompressedSize buffer := NewBuffer(4096) serializeBatch(buffer, bb.cmdSend, bb.msgMetadata, compressed) callbacks = bb.callbacks sequenceID = bb.cmdSend.Send.GetSequenceId() bb.reset() return buffer.ReadableSlice(), sequenceID, callbacks } func getCompressionProvider(compressionType pb.CompressionType) compression.Provider { switch compressionType { case pb.CompressionType_NONE: return compression.NoopProvider case pb.CompressionType_LZ4: return compression.Lz4Provider case pb.CompressionType_ZLIB: return compression.ZLibProvider case pb.CompressionType_ZSTD: return compression.ZStdProvider default: log.Panic("unsupported compression type") return nil } }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "sync" log "github.com/sirupsen/logrus" ) // BlockingQueue is a interface of block queue type BlockingQueue interface { // Put enqueue one item, block if the queue is full Put(item interface{}) // Take dequeue one item, block until it's available Take() interface{} // Poll dequeue one item, return nil if queue is empty Poll() interface{} // Peek return the first item without dequeing, return nil if queue is empty Peek() interface{} // PeekLast return last item in queue without dequeing, return nil if queue is empty PeekLast() interface{} // Size return the current size of the queue Size() int // Iterator return an iterator for the queue Iterator() BlockingQueueIterator } // BlockingQueueIterator abstract a interface of block queue iterator. type BlockingQueueIterator interface { HasNext() bool Next() interface{} } type blockingQueue struct { items []interface{} headIdx int tailIdx int size int maxSize int mutex sync.Mutex isNotEmpty *sync.Cond isNotFull *sync.Cond } type blockingQueueIterator struct { bq *blockingQueue readIdx int toRead int } // NewBlockingQueue init block queue and returns a BlockingQueue func NewBlockingQueue(maxSize int) BlockingQueue { bq := &blockingQueue{ items: make([]interface{}, maxSize), headIdx: 0, tailIdx: 0, size: 0, maxSize: maxSize, } bq.isNotEmpty = sync.NewCond(&bq.mutex) bq.isNotFull = sync.NewCond(&bq.mutex) return bq } func (bq *blockingQueue) Put(item interface{}) { bq.mutex.Lock() defer bq.mutex.Unlock() for bq.size == bq.maxSize { bq.isNotFull.Wait() } wasEmpty := bq.size == 0 bq.items[bq.tailIdx] = item bq.size++ bq.tailIdx++ if bq.tailIdx >= bq.maxSize { bq.tailIdx = 0 } if wasEmpty { // Wake up eventual reader waiting for next item bq.isNotEmpty.Signal() } } func (bq *blockingQueue) Take() interface{} { bq.mutex.Lock() defer bq.mutex.Unlock() for bq.size == 0 { bq.isNotEmpty.Wait() } return bq.dequeue() } func (bq *blockingQueue) Poll() interface{} { bq.mutex.Lock() defer bq.mutex.Unlock() if bq.size == 0 { return nil } return bq.dequeue() } func (bq *blockingQueue) Peek() interface{} { bq.mutex.Lock() defer bq.mutex.Unlock() if bq.size == 0 { return nil } return bq.items[bq.headIdx] } func (bq *blockingQueue) PeekLast() interface{} { bq.mutex.Lock() defer bq.mutex.Unlock() if bq.size == 0 { return nil } idx := (bq.headIdx + bq.size - 1) % bq.maxSize return bq.items[idx] } func (bq *blockingQueue) dequeue() interface{} { item := bq.items[bq.headIdx] bq.items[bq.headIdx] = nil bq.headIdx++ if bq.headIdx == len(bq.items) { bq.headIdx = 0 } bq.size-- bq.isNotFull.Signal() return item } func (bq *blockingQueue) Size() int { bq.mutex.Lock() defer bq.mutex.Unlock() return bq.size } func (bq *blockingQueue) Iterator() BlockingQueueIterator { bq.mutex.Lock() defer bq.mutex.Unlock() return &blockingQueueIterator{ bq: bq, readIdx: bq.headIdx, toRead: bq.size, } } func (bqi *blockingQueueIterator) HasNext() bool { return bqi.toRead > 0 } func (bqi *blockingQueueIterator) Next() interface{} { if bqi.toRead == 0 { log.Panic("Trying to read past the end of the iterator") } item := bqi.bq.items[bqi.readIdx] bqi.toRead-- bqi.readIdx++ if bqi.readIdx == bqi.bq.maxSize { bqi.readIdx = 0 } return item }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "encoding/binary" ) // Buffer is a variable-sized buffer of bytes with Read and Write methods. // The zero value for Buffer is an empty buffer ready to use. type Buffer interface { ReadableBytes() uint32 WritableBytes() uint32 // Capacity returns the capacity of the buffer's underlying byte slice, // that is, the total space allocated for the buffer's data. Capacity() uint32 IsWritable() bool Read(size uint32) []byte Get(readerIndex uint32, size uint32) []byte ReadableSlice() []byte WritableSlice() []byte // WrittenBytes advance the writer index when data was written in a slice WrittenBytes(size uint32) // MoveToFront copy the available portion of data at the beginning of the buffer MoveToFront() ReadUint16() uint16 ReadUint32() uint32 WriteUint16(n uint16) WriteUint32(n uint32) WriterIndex() uint32 ReaderIndex() uint32 Write(s []byte) Put(writerIdx uint32, s []byte) PutUint32(n uint32, writerIdx uint32) Resize(newSize uint32) // Clear will clear the current buffer data. Clear() } type buffer struct { data []byte readerIdx uint32 writerIdx uint32 } // NewBuffer creates and initializes a new Buffer using buf as its initial contents. func NewBuffer(size int) Buffer { return &buffer{ data: make([]byte, size), readerIdx: 0, writerIdx: 0, } } func NewBufferWrapper(buf []byte) Buffer { return &buffer{ data: buf, readerIdx: 0, writerIdx: uint32(len(buf)), } } func (b *buffer) ReadableBytes() uint32 { return b.writerIdx - b.readerIdx } func (b *buffer) WritableBytes() uint32 { return uint32(cap(b.data)) - b.writerIdx } func (b *buffer) Capacity() uint32 { return uint32(cap(b.data)) } func (b *buffer) IsWritable() bool { return b.WritableBytes() > 0 } func (b *buffer) Read(size uint32) []byte { res := b.data[b.readerIdx : b.readerIdx+size] b.readerIdx += size return res } func (b *buffer) Get(readerIdx uint32, size uint32) []byte { return b.data[readerIdx : readerIdx+size] } func (b *buffer) ReadableSlice() []byte { return b.data[b.readerIdx:b.writerIdx] } func (b *buffer) WritableSlice() []byte { return b.data[b.writerIdx:] } func (b *buffer) WrittenBytes(size uint32) { b.writerIdx += size } func (b *buffer) WriterIndex() uint32 { return b.writerIdx } func (b *buffer) ReaderIndex() uint32 { return b.readerIdx } func (b *buffer) MoveToFront() { size := b.ReadableBytes() copy(b.data, b.Read(size)) b.readerIdx = 0 b.writerIdx = size } func (b *buffer) Resize(newSize uint32) { newData := make([]byte, newSize) size := b.ReadableBytes() copy(newData, b.Read(size)) b.data = newData b.readerIdx = 0 b.writerIdx = size } func (b *buffer) resizeIfNeeded(spaceNeeded int) { if b.WritableBytes() < uint32(spaceNeeded) { capacityNeeded := uint32(cap(b.data) + spaceNeeded) minCapacityIncrease := uint32(cap(b.data) * 3 / 2) if capacityNeeded < minCapacityIncrease { capacityNeeded = minCapacityIncrease } b.Resize(capacityNeeded) } } func (b *buffer) ReadUint32() uint32 { return binary.BigEndian.Uint32(b.Read(4)) } func (b *buffer) ReadUint16() uint16 { return binary.BigEndian.Uint16(b.Read(2)) } func (b *buffer) WriteUint32(n uint32) { b.resizeIfNeeded(4) binary.BigEndian.PutUint32(b.WritableSlice(), n) b.writerIdx += 4 } func (b *buffer) PutUint32(n uint32, idx uint32) { binary.BigEndian.PutUint32(b.data[idx:], n) } func (b *buffer) WriteUint16(n uint16) { b.resizeIfNeeded(2) binary.BigEndian.PutUint16(b.WritableSlice(), n) b.writerIdx += 2 } func (b *buffer) Write(s []byte) { b.resizeIfNeeded(len(s)) copy(b.WritableSlice(), s) b.writerIdx += uint32(len(s)) } func (b *buffer) Put(writerIdx uint32, s []byte) { copy(b.data[writerIdx:], s) } func (b *buffer) Clear() { b.readerIdx = 0 b.writerIdx = 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 internal import ( "hash" "hash/crc32" ) // crc32cTable holds the precomputed crc32 hash table // used by Pulsar (crc32c) var crc32cTable = crc32.MakeTable(crc32.Castagnoli) type CheckSum struct { hash hash.Hash } // Crc32cCheckSum handles computing the checksum. func Crc32cCheckSum(data []byte) uint32 { return crc32.Checksum(data, crc32cTable) } func (cs *CheckSum) Write(p []byte) (int, error) { if cs.hash == nil { cs.hash = crc32.New(crc32cTable) } return cs.hash.Write(p) } func (cs *CheckSum) compute() []byte { if cs.hash == nil { return nil } return cs.hash.Sum(nil) }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 "sync" // ClientHandlerMap is a simple concurrent-safe map for the client type type ClientHandlers struct { handlers map[Closable]bool l *sync.RWMutex } func NewClientHandlers() ClientHandlers { return ClientHandlers{ handlers: map[Closable]bool{}, l: &sync.RWMutex{}, } } func (h *ClientHandlers) Add(c Closable) { h.l.Lock() defer h.l.Unlock() h.handlers[c] = true } func (h *ClientHandlers) Del(c Closable) { h.l.Lock() defer h.l.Unlock() delete(h.handlers, c) } func (h *ClientHandlers) Val(c Closable) bool { h.l.RLock() defer h.l.RUnlock() return h.handlers[c] } func (h *ClientHandlers) Close() { h.l.Lock() handlers := make([]Closable, 0, len(h.handlers)) for handler := range h.handlers { handlers = append(handlers, handler) } h.l.Unlock() for _, handler := range handlers { handler.Close() } }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "errors" "fmt" "github.com/golang/protobuf/proto" log "github.com/sirupsen/logrus" "github.com/apache/pulsar-client-go/pulsar/internal/pb" ) const ( // MaxFrameSize limit the maximum size that pulsar allows for messages to be sent. MaxFrameSize = 5 * 1024 * 1024 magicCrc32c uint16 = 0x0e01 ) // ErrCorruptedMessage is the error returned by ReadMessageData when it has detected corrupted data. // The data is considered corrupted if it's missing a header, a checksum mismatch or there // was an error when unmarshalling the message metadata. var ErrCorruptedMessage = errors.New("corrupted message") // ErrEOM is the error returned by ReadMessage when no more input is available. var ErrEOM = errors.New("EOF") func NewMessageReader(headersAndPayload Buffer) *MessageReader { return &MessageReader{ buffer: headersAndPayload, } } func NewMessageReaderFromArray(headersAndPayload []byte) *MessageReader { return NewMessageReader(NewBufferWrapper(headersAndPayload)) } // MessageReader provides helper methods to parse // the metadata and messages from the binary format // Wire format for a messages // // Old format (single message) // [MAGIC_NUMBER][CHECKSUM] [METADATA_SIZE][METADATA] [PAYLOAD] // // Batch format // [MAGIC_NUMBER][CHECKSUM] [METADATA_SIZE][METADATA] [METADATA_SIZE][METADATA][PAYLOAD] // [METADATA_SIZE][METADATA][PAYLOAD] // type MessageReader struct { buffer Buffer // true if we are parsing a batched message - set after parsing the message metadata batched bool } // ReadChecksum func (r *MessageReader) readChecksum() (uint32, error) { if r.buffer.ReadableBytes() < 6 { return 0, errors.New("missing message header") } // reader magic number magicNumber := r.buffer.ReadUint16() if magicNumber != magicCrc32c { return 0, ErrCorruptedMessage } checksum := r.buffer.ReadUint32() return checksum, nil } func (r *MessageReader) ReadMessageMetadata() (*pb.MessageMetadata, error) { // Wire format // [MAGIC_NUMBER][CHECKSUM] [METADATA_SIZE][METADATA] // read checksum checksum, err := r.readChecksum() if err != nil { return nil, err } // validate checksum computedChecksum := Crc32cCheckSum(r.buffer.ReadableSlice()) if checksum != computedChecksum { return nil, fmt.Errorf("checksum mismatch received: 0x%x computed: 0x%x", checksum, computedChecksum) } size := r.buffer.ReadUint32() data := r.buffer.Read(size) var meta pb.MessageMetadata if err := proto.Unmarshal(data, &meta); err != nil { return nil, ErrCorruptedMessage } if meta.NumMessagesInBatch != nil { r.batched = true } return &meta, nil } func (r *MessageReader) ReadMessage() (*pb.SingleMessageMetadata, []byte, error) { if r.buffer.ReadableBytes() == 0 && r.buffer.Capacity() > 0 { return nil, nil, ErrEOM } if !r.batched { return r.readMessage() } return r.readSingleMessage() } func (r *MessageReader) readMessage() (*pb.SingleMessageMetadata, []byte, error) { // Wire format // [PAYLOAD] return nil, r.buffer.Read(r.buffer.ReadableBytes()), nil } func (r *MessageReader) readSingleMessage() (*pb.SingleMessageMetadata, []byte, error) { // Wire format // [METADATA_SIZE][METADATA][PAYLOAD] size := r.buffer.ReadUint32() var meta pb.SingleMessageMetadata if err := proto.Unmarshal(r.buffer.Read(size), &meta); err != nil { return nil, nil, err } return &meta, r.buffer.Read(uint32(meta.GetPayloadSize())), nil } func (r *MessageReader) ResetBuffer(buffer Buffer) { r.buffer = buffer } func baseCommand(cmdType pb.BaseCommand_Type, msg proto.Message) *pb.BaseCommand { cmd := &pb.BaseCommand{ Type: &cmdType, } switch cmdType { case pb.BaseCommand_CONNECT: cmd.Connect = msg.(*pb.CommandConnect) case pb.BaseCommand_LOOKUP: cmd.LookupTopic = msg.(*pb.CommandLookupTopic) case pb.BaseCommand_PARTITIONED_METADATA: cmd.PartitionMetadata = msg.(*pb.CommandPartitionedTopicMetadata) case pb.BaseCommand_PRODUCER: cmd.Producer = msg.(*pb.CommandProducer) case pb.BaseCommand_SUBSCRIBE: cmd.Subscribe = msg.(*pb.CommandSubscribe) case pb.BaseCommand_FLOW: cmd.Flow = msg.(*pb.CommandFlow) case pb.BaseCommand_PING: cmd.Ping = msg.(*pb.CommandPing) case pb.BaseCommand_PONG: cmd.Pong = msg.(*pb.CommandPong) case pb.BaseCommand_SEND: cmd.Send = msg.(*pb.CommandSend) case pb.BaseCommand_CLOSE_PRODUCER: cmd.CloseProducer = msg.(*pb.CommandCloseProducer) case pb.BaseCommand_CLOSE_CONSUMER: cmd.CloseConsumer = msg.(*pb.CommandCloseConsumer) case pb.BaseCommand_ACK: cmd.Ack = msg.(*pb.CommandAck) case pb.BaseCommand_SEEK: cmd.Seek = msg.(*pb.CommandSeek) case pb.BaseCommand_UNSUBSCRIBE: cmd.Unsubscribe = msg.(*pb.CommandUnsubscribe) case pb.BaseCommand_REDELIVER_UNACKNOWLEDGED_MESSAGES: cmd.RedeliverUnacknowledgedMessages = msg.(*pb.CommandRedeliverUnacknowledgedMessages) case pb.BaseCommand_GET_TOPICS_OF_NAMESPACE: cmd.GetTopicsOfNamespace = msg.(*pb.CommandGetTopicsOfNamespace) case pb.BaseCommand_GET_LAST_MESSAGE_ID: cmd.GetLastMessageId = msg.(*pb.CommandGetLastMessageId) case pb.BaseCommand_AUTH_RESPONSE: cmd.AuthResponse = msg.(*pb.CommandAuthResponse) default: log.Panic("Missing command type: ", cmdType) } return cmd } func addSingleMessageToBatch(wb Buffer, smm proto.Message, payload []byte) { serialized, err := proto.Marshal(smm) if err != nil { log.WithError(err).Fatal("Protobuf serialization error") } wb.WriteUint32(uint32(len(serialized))) wb.Write(serialized) wb.Write(payload) } func serializeBatch(wb Buffer, cmdSend proto.Message, msgMetadata proto.Message, payload []byte) { // Wire format // [TOTAL_SIZE] [CMD_SIZE][CMD] [MAGIC_NUMBER][CHECKSUM] [METADATA_SIZE][METADATA] [PAYLOAD] cmdSize := proto.Size(cmdSend) msgMetadataSize := proto.Size(msgMetadata) payloadSize := len(payload) magicAndChecksumLength := 2 + 4 /* magic + checksumLength */ headerContentSize := 4 + cmdSize + magicAndChecksumLength + 4 + msgMetadataSize // cmdLength + cmdSize + magicLength + checksumSize + msgMetadataLength + msgMetadataSize totalSize := headerContentSize + payloadSize wb.WriteUint32(uint32(totalSize)) // External frame // Write cmd wb.WriteUint32(uint32(cmdSize)) serialized, err := proto.Marshal(cmdSend) if err != nil { log.WithError(err).Fatal("Protobuf error when serializing cmdSend") } wb.Write(serialized) // Create checksum placeholder wb.WriteUint16(magicCrc32c) checksumIdx := wb.WriterIndex() wb.WriteUint32(0) // skip 4 bytes of checksum // Write metadata metadataStartIdx := wb.WriterIndex() wb.WriteUint32(uint32(msgMetadataSize)) serialized, err = proto.Marshal(msgMetadata) if err != nil { log.WithError(err).Fatal("Protobuf error when serializing msgMetadata") } wb.Write(serialized) wb.Write(payload) // Write checksum at created checksum-placeholder endIdx := wb.WriterIndex() checksum := Crc32cCheckSum(wb.Get(metadataStartIdx, endIdx-metadataStartIdx)) // set computed checksum wb.PutUint32(checksum, checksumIdx) } // ConvertFromStringMap convert a string map to a KeyValue []byte func ConvertFromStringMap(m map[string]string) []*pb.KeyValue { list := make([]*pb.KeyValue, len(m)) i := 0 for k, v := range m { list[i] = &pb.KeyValue{ Key: proto.String(k), Value: proto.String(v), } i++ } return list } // ConvertToStringMap convert a KeyValue []byte to string map func ConvertToStringMap(pbb []*pb.KeyValue) map[string]string { m := make(map[string]string) for _, kv := range pbb { m[*kv.Key] = *kv.Value } return m }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 compression import ( "github.com/pierrec/lz4" ) type lz4Provider struct{} // NewLz4Provider return a interface of Provider. func NewLz4Provider() Provider { return &lz4Provider{} } func (lz4Provider) CanCompress() bool { return true } func (lz4Provider) Compress(data []byte) []byte { const tableSize = 1 << 16 hashTable := make([]int, tableSize) maxSize := lz4.CompressBlockBound(len(data)) compressed := make([]byte, maxSize) size, err := lz4.CompressBlock(data, compressed, hashTable) if err != nil { panic("Failed to compress") } if size == 0 { // The data block was not compressed. Just repeat it with // the block header flag to signal it's not compressed headerSize := writeSize(len(data), compressed) copy(compressed[headerSize:], data) return compressed[:len(data)+headerSize] } return compressed[:size] } // Write the encoded size for the uncompressed payload func writeSize(size int, dst []byte) int { if size < 0xF { dst[0] |= byte(size << 4) return 1 } dst[0] |= 0xF0 l := size - 0xF i := 1 for ; l >= 0xFF; l -= 0xFF { dst[i] = 0xFF i++ } dst[i] = byte(l) return i + 1 } func (lz4Provider) Decompress(compressedData []byte, originalSize int) ([]byte, error) { uncompressed := make([]byte, originalSize) _, err := lz4.UncompressBlock(compressedData, uncompressed) return uncompressed, err }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 compression type noopProvider struct{} // NewNoopProvider returns a Provider interface func NewNoopProvider() Provider { return &noopProvider{} } // CanCompress always returns true, in the case of noopProvider, noopProvider means no compression. func (noopProvider) CanCompress() bool { return true } func (noopProvider) Compress(data []byte) []byte { return data } func (noopProvider) Decompress(compressedData []byte, originalSize int) ([]byte, error) { return compressedData, nil }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 compression import ( "bytes" "compress/zlib" "io" ) type zlibProvider struct{} // NewZLibProvider returns a Provider interface func NewZLibProvider() Provider { return &zlibProvider{} } func (zlibProvider) CanCompress() bool { return true } func (zlibProvider) Compress(data []byte) []byte { var b bytes.Buffer w := zlib.NewWriter(&b) if _, err := w.Write(data); err != nil { return nil } if err := w.Close(); err != nil { return nil } return b.Bytes() } func (zlibProvider) Decompress(compressedData []byte, originalSize int) ([]byte, error) { r, err := zlib.NewReader(bytes.NewReader(compressedData)) if err != nil { return nil, err } uncompressed := make([]byte, originalSize) if _, err = io.ReadFull(r, uncompressed); err != nil { return nil, err } if err = r.Close(); err != nil { return nil, err } return uncompressed, nil }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 compression import ( "github.com/klauspost/compress/zstd" "github.com/pkg/errors" ) type zstdProvider struct { encoder *zstd.Encoder decoder *zstd.Decoder } func NewZStdProvider() Provider { p := &zstdProvider{} p.encoder, _ = zstd.NewWriter(nil) p.decoder, _ = zstd.NewReader(nil) return p } func (p *zstdProvider) CanCompress() bool { return true } func (p *zstdProvider) Compress(data []byte) []byte { return p.encoder.EncodeAll(data, []byte{}) } func (p *zstdProvider) Decompress(compressedData []byte, originalSize int) (dst []byte, err error) { dst, err = p.decoder.DecodeAll(compressedData, nil) if err == nil && len(dst) != originalSize { return nil, errors.New("Invalid uncompressed size") } return }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "crypto/tls" "crypto/x509" "errors" "fmt" "io/ioutil" "net" "net/url" "sync" "sync/atomic" "time" "github.com/golang/protobuf/proto" log "github.com/sirupsen/logrus" "github.com/apache/pulsar-client-go/pulsar/internal/auth" "github.com/apache/pulsar-client-go/pulsar/internal/pb" ) const ( // TODO: Find a better way to embed the version in the library code PulsarVersion = "0.1" ClientVersionString = "Pulsar Go " + PulsarVersion PulsarProtocolVersion = int32(pb.ProtocolVersion_v13) ) type TLSOptions struct { TrustCertsFilePath string AllowInsecureConnection bool ValidateHostname bool } // ConnectionListener is a user of a connection (eg. a producer or // a consumer) that can register itself to get notified // when the connection is closed. type ConnectionListener interface { // ReceivedSendReceipt receive and process the return value of the send command. ReceivedSendReceipt(response *pb.CommandSendReceipt) // ConnectionClosed close the TCP connection. ConnectionClosed() } // Connection is a interface of client cnx. type Connection interface { SendRequest(requestID uint64, req *pb.BaseCommand, callback func(*pb.BaseCommand, error)) SendRequestNoWait(req *pb.BaseCommand) WriteData(data []byte) RegisterListener(id uint64, listener ConnectionListener) UnregisterListener(id uint64) AddConsumeHandler(id uint64, handler ConsumerHandler) DeleteConsumeHandler(id uint64) ID() string Close() } type ConsumerHandler interface { MessageReceived(response *pb.CommandMessage, headersAndPayload Buffer) error // ConnectionClosed close the TCP connection. ConnectionClosed() } type connectionState int const ( connectionInit connectionState = iota connectionConnecting connectionTCPConnected connectionReady connectionClosed ) func (s connectionState) String() string { switch s { case connectionInit: return "Initializing" case connectionConnecting: return "Connecting" case connectionTCPConnected: return "TCPConnected" case connectionReady: return "Ready" case connectionClosed: return "Closed" default: return "Unknown" } } const keepAliveInterval = 30 * time.Second type request struct { id *uint64 cmd *pb.BaseCommand callback func(command *pb.BaseCommand, err error) } type incomingCmd struct { cmd *pb.BaseCommand headersAndPayload Buffer } type connection struct { sync.Mutex cond *sync.Cond state connectionState connectionTimeout time.Duration logicalAddr *url.URL physicalAddr *url.URL cnx net.Conn writeBufferLock sync.Mutex writeBuffer Buffer reader *connectionReader lastDataReceivedLock sync.Mutex lastDataReceivedTime time.Time pingTicker *time.Ticker pingCheckTicker *time.Ticker log *log.Entry requestIDGenerator uint64 incomingRequestsCh chan *request incomingCmdCh chan *incomingCmd closeCh chan interface{} writeRequestsCh chan []byte pendingReqs map[uint64]*request listeners map[uint64]ConnectionListener consumerHandlersLock sync.RWMutex consumerHandlers map[uint64]ConsumerHandler tlsOptions *TLSOptions auth auth.Provider } func newConnection(logicalAddr *url.URL, physicalAddr *url.URL, tlsOptions *TLSOptions, connectionTimeout time.Duration, auth auth.Provider) *connection { cnx := &connection{ state: connectionInit, connectionTimeout: connectionTimeout, logicalAddr: logicalAddr, physicalAddr: physicalAddr, writeBuffer: NewBuffer(4096), log: log.WithField("remote_addr", physicalAddr), pendingReqs: make(map[uint64]*request), lastDataReceivedTime: time.Now(), pingTicker: time.NewTicker(keepAliveInterval), pingCheckTicker: time.NewTicker(keepAliveInterval), tlsOptions: tlsOptions, auth: auth, closeCh: make(chan interface{}), incomingRequestsCh: make(chan *request, 10), incomingCmdCh: make(chan *incomingCmd, 10), writeRequestsCh: make(chan []byte, 10), listeners: make(map[uint64]ConnectionListener), consumerHandlers: make(map[uint64]ConsumerHandler), } cnx.reader = newConnectionReader(cnx) cnx.cond = sync.NewCond(cnx) return cnx } func (c *connection) start() { // Each connection gets its own goroutine that will go func() { if c.connect() { if c.doHandshake() { c.run() } else { c.changeState(connectionClosed) } } else { c.changeState(connectionClosed) } }() } func (c *connection) connect() bool { c.log.Info("Connecting to broker") var ( err error cnx net.Conn tlsConfig *tls.Config ) if c.tlsOptions == nil { // Clear text connection cnx, err = net.DialTimeout("tcp", c.physicalAddr.Host, c.connectionTimeout) } else { // TLS connection tlsConfig, err = c.getTLSConfig() if err != nil { c.log.WithError(err).Warn("Failed to configure TLS ") return false } d := &net.Dialer{Timeout: c.connectionTimeout} cnx, err = tls.DialWithDialer(d, "tcp", c.physicalAddr.Host, tlsConfig) } if err != nil { c.log.WithError(err).Warn("Failed to connect to broker.") c.Close() return false } c.Lock() c.cnx = cnx c.log = c.log.WithField("local_addr", c.cnx.LocalAddr()) c.log.Info("TCP connection established") c.Unlock() c.changeState(connectionTCPConnected) return true } func (c *connection) doHandshake() bool { // Send 'Connect' command to initiate handshake authData, err := c.auth.GetData() if err != nil { c.log.WithError(err).Warn("Failed to load auth credentials") return false } // During the initial handshake, the internal keep alive is not // active yet, so we need to timeout write and read requests c.cnx.SetDeadline(time.Now().Add(keepAliveInterval)) cmdConnect := &pb.CommandConnect{ ProtocolVersion: proto.Int32(PulsarProtocolVersion), ClientVersion: proto.String(ClientVersionString), AuthMethodName: proto.String(c.auth.Name()), AuthData: authData, FeatureFlags: &pb.FeatureFlags{ SupportsAuthRefresh: proto.Bool(true), }, } if c.logicalAddr.Host != c.physicalAddr.Host { cmdConnect.ProxyToBrokerUrl = proto.String(c.logicalAddr.Host) } c.writeCommand(baseCommand(pb.BaseCommand_CONNECT, cmdConnect)) cmd, _, err := c.reader.readSingleCommand() if err != nil { c.log.WithError(err).Warn("Failed to perform initial handshake") return false } // Reset the deadline so that we don't use read timeouts c.cnx.SetDeadline(time.Time{}) if cmd.Connected == nil { c.log.Warnf("Failed to perform initial handshake - Expecting 'Connected' cmd, got '%s'", cmd.Type) return false } c.log.Info("Connection is ready") c.changeState(connectionReady) return true } func (c *connection) waitUntilReady() error { c.Lock() defer c.Unlock() for c.state != connectionReady { c.log.Debug("Wait until connection is ready. State: ", c.state) if c.state == connectionClosed { return errors.New("connection error") } // wait for a new connection state change c.cond.Wait() } return nil } func (c *connection) run() { // All reads come from the reader goroutine go c.reader.readFromConnection() go c.runPingCheck() for { select { case <-c.closeCh: c.Close() return case req := <-c.incomingRequestsCh: if req == nil { return } c.internalSendRequest(req) case cmd := <-c.incomingCmdCh: c.internalReceivedCommand(cmd.cmd, cmd.headersAndPayload) case data := <-c.writeRequestsCh: if data == nil { return } c.internalWriteData(data) case <-c.pingTicker.C: c.sendPing() } } } func (c *connection) runPingCheck() { for { select { case <-c.closeCh: return case <-c.pingCheckTicker.C: if c.lastDataReceived().Add(2 * keepAliveInterval).Before(time.Now()) { // We have not received a response to the previous Ping request, the // connection to broker is stale c.log.Warn("Detected stale connection to broker") c.TriggerClose() return } } } } func (c *connection) WriteData(data []byte) { c.writeRequestsCh <- data } func (c *connection) internalWriteData(data []byte) { c.log.Debug("Write data: ", len(data)) if _, err := c.cnx.Write(data); err != nil { c.log.WithError(err).Warn("Failed to write on connection") c.Close() } } func (c *connection) writeCommand(cmd proto.Message) { // Wire format // [FRAME_SIZE] [CMD_SIZE][CMD] cmdSize := uint32(proto.Size(cmd)) frameSize := cmdSize + 4 c.writeBufferLock.Lock() defer c.writeBufferLock.Unlock() c.writeBuffer.Clear() c.writeBuffer.WriteUint32(frameSize) c.writeBuffer.WriteUint32(cmdSize) serialized, err := proto.Marshal(cmd) if err != nil { c.log.WithError(err).Fatal("Protobuf serialization error") } c.writeBuffer.Write(serialized) data := c.writeBuffer.ReadableSlice() c.internalWriteData(data) } func (c *connection) receivedCommand(cmd *pb.BaseCommand, headersAndPayload Buffer) { c.incomingCmdCh <- &incomingCmd{cmd, headersAndPayload} } func (c *connection) internalReceivedCommand(cmd *pb.BaseCommand, headersAndPayload Buffer) { c.log.Debugf("Received command: %s -- payload: %v", cmd, headersAndPayload) c.setLastDataReceived(time.Now()) switch *cmd.Type { case pb.BaseCommand_SUCCESS: c.handleResponse(cmd.Success.GetRequestId(), cmd) case pb.BaseCommand_PRODUCER_SUCCESS: c.handleResponse(cmd.ProducerSuccess.GetRequestId(), cmd) case pb.BaseCommand_PARTITIONED_METADATA_RESPONSE: c.handleResponse(cmd.PartitionMetadataResponse.GetRequestId(), cmd) case pb.BaseCommand_LOOKUP_RESPONSE: lookupResult := cmd.LookupTopicResponse c.handleResponse(lookupResult.GetRequestId(), cmd) case pb.BaseCommand_CONSUMER_STATS_RESPONSE: c.handleResponse(cmd.ConsumerStatsResponse.GetRequestId(), cmd) case pb.BaseCommand_GET_LAST_MESSAGE_ID_RESPONSE: c.handleResponse(cmd.GetLastMessageIdResponse.GetRequestId(), cmd) case pb.BaseCommand_GET_TOPICS_OF_NAMESPACE_RESPONSE: c.handleResponse(cmd.GetTopicsOfNamespaceResponse.GetRequestId(), cmd) case pb.BaseCommand_GET_SCHEMA_RESPONSE: c.handleResponse(cmd.GetSchemaResponse.GetRequestId(), cmd) case pb.BaseCommand_ERROR: c.handleResponseError(cmd.GetError()) case pb.BaseCommand_CLOSE_PRODUCER: c.handleCloseProducer(cmd.GetCloseProducer()) case pb.BaseCommand_CLOSE_CONSUMER: c.handleCloseConsumer(cmd.GetCloseConsumer()) case pb.BaseCommand_AUTH_CHALLENGE: c.handleAuthChallenge(cmd.GetAuthChallenge()) case pb.BaseCommand_SEND_RECEIPT: c.handleSendReceipt(cmd.GetSendReceipt()) case pb.BaseCommand_SEND_ERROR: case pb.BaseCommand_MESSAGE: c.handleMessage(cmd.GetMessage(), headersAndPayload) case pb.BaseCommand_PING: c.handlePing() case pb.BaseCommand_PONG: c.handlePong() case pb.BaseCommand_ACTIVE_CONSUMER_CHANGE: default: c.log.Errorf("Received invalid command type: %s", cmd.Type) c.Close() } } func (c *connection) Write(data []byte) { c.writeRequestsCh <- data } func (c *connection) SendRequest(requestID uint64, req *pb.BaseCommand, callback func(command *pb.BaseCommand, err error)) { c.incomingRequestsCh <- &request{ id: &requestID, cmd: req, callback: callback, } } func (c *connection) SendRequestNoWait(req *pb.BaseCommand) { c.incomingRequestsCh <- &request{ id: nil, cmd: req, callback: nil, } } func (c *connection) internalSendRequest(req *request) { if req.id != nil { c.pendingReqs[*req.id] = req } c.writeCommand(req.cmd) } func (c *connection) handleResponse(requestID uint64, response *pb.BaseCommand) { request, ok := c.pendingReqs[requestID] if !ok { c.log.Warnf("Received unexpected response for request %d of type %s", requestID, response.Type) return } delete(c.pendingReqs, requestID) request.callback(response, nil) } func (c *connection) handleResponseError(serverError *pb.CommandError) { requestID := serverError.GetRequestId() request, ok := c.pendingReqs[requestID] if !ok { c.log.Warnf("Received unexpected error response for request %d of type %s", requestID, serverError.GetError()) return } delete(c.pendingReqs, requestID) errMsg := fmt.Sprintf("server error: %s: %s", serverError.GetError(), serverError.GetMessage()) request.callback(nil, errors.New(errMsg)) } func (c *connection) handleSendReceipt(response *pb.CommandSendReceipt) { producerID := response.GetProducerId() c.Lock() defer c.Unlock() if producer, ok := c.listeners[producerID]; ok { producer.ReceivedSendReceipt(response) } else { c.log.WithField("producerID", producerID).Warn("Got unexpected send receipt for message: ", response.MessageId) } } func (c *connection) handleMessage(response *pb.CommandMessage, payload Buffer) { c.log.Debug("Got Message: ", response) consumerID := response.GetConsumerId() if consumer, ok := c.consumerHandler(consumerID); ok { err := consumer.MessageReceived(response, payload) if err != nil { c.log.WithField("consumerID", consumerID).WithError(err).Error("handle message Id: ", response.MessageId) } } else { c.log.WithField("consumerID", consumerID).Warn("Got unexpected message: ", response.MessageId) } } func (c *connection) lastDataReceived() time.Time { c.lastDataReceivedLock.Lock() defer c.lastDataReceivedLock.Unlock() t := c.lastDataReceivedTime return t } func (c *connection) setLastDataReceived(t time.Time) { c.lastDataReceivedLock.Lock() defer c.lastDataReceivedLock.Unlock() c.lastDataReceivedTime = t } func (c *connection) sendPing() { c.log.Debug("Sending PING") c.writeCommand(baseCommand(pb.BaseCommand_PING, &pb.CommandPing{})) } func (c *connection) handlePong() { c.log.Debug("Received PONG response") } func (c *connection) handlePing() { c.log.Debug("Responding to PING request") c.writeCommand(baseCommand(pb.BaseCommand_PONG, &pb.CommandPong{})) } func (c *connection) handleAuthChallenge(authChallenge *pb.CommandAuthChallenge) { c.log.Debugf("Received auth challenge from broker: %s", authChallenge.GetChallenge().GetAuthMethodName()) // Get new credentials from the provider authData, err := c.auth.GetData() if err != nil { c.log.WithError(err).Warn("Failed to load auth credentials") c.Close() return } cmdAuthResponse := &pb.CommandAuthResponse{ ProtocolVersion: proto.Int32(PulsarProtocolVersion), ClientVersion: proto.String(ClientVersionString), Response: &pb.AuthData{ AuthMethodName: proto.String(c.auth.Name()), AuthData: authData, }, } c.writeCommand(baseCommand(pb.BaseCommand_AUTH_RESPONSE, cmdAuthResponse)) } func (c *connection) handleCloseConsumer(closeConsumer *pb.CommandCloseConsumer) { c.log.Infof("Broker notification of Closed consumer: %d", closeConsumer.GetConsumerId()) consumerID := closeConsumer.GetConsumerId() c.Lock() defer c.Unlock() if consumer, ok := c.consumerHandler(consumerID); ok { consumer.ConnectionClosed() delete(c.listeners, consumerID) } else { c.log.WithField("consumerID", consumerID).Warnf("Consumer with ID not found while closing consumer") } } func (c *connection) handleCloseProducer(closeProducer *pb.CommandCloseProducer) { c.log.Infof("Broker notification of Closed producer: %d", closeProducer.GetProducerId()) producerID := closeProducer.GetProducerId() c.Lock() defer c.Unlock() if producer, ok := c.listeners[producerID]; ok { producer.ConnectionClosed() delete(c.listeners, producerID) } else { c.log.WithField("producerID", producerID).Warn("Producer with ID not found while closing producer") } } func (c *connection) RegisterListener(id uint64, listener ConnectionListener) { c.Lock() defer c.Unlock() c.listeners[id] = listener } func (c *connection) UnregisterListener(id uint64) { c.Lock() defer c.Unlock() delete(c.listeners, id) } // Triggers the connection close by forcing the socket to close and // broadcasting the notification on the close channel func (c *connection) TriggerClose() { cnx := c.cnx if cnx != nil { cnx.Close() } select { case <-c.closeCh: return default: close(c.closeCh) } } func (c *connection) Close() { c.Lock() defer c.Unlock() c.cond.Broadcast() if c.state == connectionClosed { return } c.log.Info("Connection closed") c.state = connectionClosed if c.cnx != nil { c.cnx.Close() } c.pingTicker.Stop() c.pingCheckTicker.Stop() for _, listener := range c.listeners { listener.ConnectionClosed() } for _, req := range c.pendingReqs { req.callback(nil, errors.New("connection closed")) } consumerHandlers := make(map[uint64]ConsumerHandler) c.consumerHandlersLock.RLock() for id, handler := range c.consumerHandlers { consumerHandlers[id] = handler } c.consumerHandlersLock.RUnlock() for _, handler := range consumerHandlers { handler.ConnectionClosed() } } func (c *connection) changeState(state connectionState) { c.Lock() c.state = state c.cond.Broadcast() c.Unlock() } func (c *connection) newRequestID() uint64 { return atomic.AddUint64(&c.requestIDGenerator, 1) } func (c *connection) 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") } } if c.tlsOptions.ValidateHostname { tlsConfig.ServerName = c.physicalAddr.Hostname() } cert, err := c.auth.GetTLSCertificate() if err != nil { return nil, err } if cert != nil { tlsConfig.Certificates = []tls.Certificate{*cert} } return tlsConfig, nil } func (c *connection) AddConsumeHandler(id uint64, handler ConsumerHandler) { c.consumerHandlersLock.Lock() defer c.consumerHandlersLock.Unlock() c.consumerHandlers[id] = handler } func (c *connection) DeleteConsumeHandler(id uint64) { c.consumerHandlersLock.Lock() defer c.consumerHandlersLock.Unlock() delete(c.consumerHandlers, id) } func (c *connection) consumerHandler(id uint64) (ConsumerHandler, bool) { c.consumerHandlersLock.RLock() defer c.consumerHandlersLock.RUnlock() h, ok := c.consumerHandlers[id] return h, ok } func (c *connection) ID() string { return fmt.Sprintf("%s -> %s", c.cnx.LocalAddr(), c.cnx.RemoteAddr()) }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "net/url" "sync" "time" "github.com/apache/pulsar-client-go/pulsar/internal/auth" log "github.com/sirupsen/logrus" ) // ConnectionPool is a interface of connection pool. type ConnectionPool interface { // GetConnection get a connection from ConnectionPool. GetConnection(logicalAddr *url.URL, physicalAddr *url.URL) (Connection, error) // Close all the connections in the pool Close() } type connectionPool struct { pool sync.Map connectionTimeout time.Duration tlsOptions *TLSOptions auth auth.Provider } // NewConnectionPool init connection pool. func NewConnectionPool(tlsOptions *TLSOptions, auth auth.Provider, connectionTimeout time.Duration) ConnectionPool { return &connectionPool{ tlsOptions: tlsOptions, auth: auth, connectionTimeout: connectionTimeout, } } func (p *connectionPool) GetConnection(logicalAddr *url.URL, physicalAddr *url.URL) (Connection, error) { cachedCnx, found := p.pool.Load(logicalAddr.Host) if found { cnx := cachedCnx.(*connection) log.Debug("Found connection in cache:", cnx.logicalAddr, cnx.physicalAddr) if err := cnx.waitUntilReady(); err == nil { // Connection is ready to be used return cnx, nil } // The cached connection is failed p.pool.Delete(logicalAddr.Host) log.Debug("Removed failed connection from pool:", cnx.logicalAddr, cnx.physicalAddr) } // Try to create a new connection newConnection := newConnection(logicalAddr, physicalAddr, p.tlsOptions, p.connectionTimeout, p.auth) newCnx, wasCached := p.pool.LoadOrStore(logicalAddr.Host, newConnection) cnx := newCnx.(*connection) if !wasCached { cnx.start() } else { newConnection.Close() } if err := cnx.waitUntilReady(); err != nil { return nil, err } return cnx, nil } func (p *connectionPool) Close() { p.pool.Range(func(key, value interface{}) bool { value.(Connection).Close() return true }) }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "bufio" "io" "github.com/apache/pulsar-client-go/pulsar/internal/pb" "github.com/golang/protobuf/proto" "github.com/pkg/errors" ) type connectionReader struct { cnx *connection buffer Buffer reader *bufio.Reader } func newConnectionReader(cnx *connection) *connectionReader { return &connectionReader{ cnx: cnx, reader: bufio.NewReader(cnx.cnx), buffer: NewBuffer(4096), } } func (r *connectionReader) readFromConnection() { for { cmd, headersAndPayload, err := r.readSingleCommand() if err != nil { r.cnx.log.WithError(err).Info("Error reading from connection") r.cnx.TriggerClose() break } // Process var payloadLen uint32 if headersAndPayload != nil { payloadLen = headersAndPayload.ReadableBytes() } r.cnx.log.Debug("Got command! ", cmd, " with payload size: ", payloadLen) r.cnx.receivedCommand(cmd, headersAndPayload) } } func (r *connectionReader) readSingleCommand() (cmd *pb.BaseCommand, headersAndPayload Buffer, err error) { // First, we need to read the frame size if r.buffer.ReadableBytes() < 4 { if r.buffer.ReadableBytes() == 0 { // If the buffer is empty, just go back to write at the beginning r.buffer.Clear() } if err := r.readAtLeast(4); err != nil { return nil, nil, errors.Errorf("Short read when reading frame size: %s", err) } } // We have enough to read frame size frameSize := r.buffer.ReadUint32() if frameSize > MaxFrameSize { r.cnx.log.Warnf("Received too big frame size. size=%d", frameSize) r.cnx.TriggerClose() return nil, nil, errors.New("Frame size too big") } // Next, we read the rest of the frame if r.buffer.ReadableBytes() < frameSize { remainingBytes := frameSize - r.buffer.ReadableBytes() if err := r.readAtLeast(remainingBytes); err != nil { return nil, nil, errors.Errorf("Short read when reading frame: %s", err) } } // We have now the complete frame cmdSize := r.buffer.ReadUint32() cmd, err = r.deserializeCmd(r.buffer.Read(cmdSize)) if err != nil { return nil, nil, err } // Also read the eventual payload headersAndPayloadSize := frameSize - (cmdSize + 4) if cmdSize+4 < frameSize { headersAndPayload = NewBuffer(int(headersAndPayloadSize)) headersAndPayload.Write(r.buffer.Read(headersAndPayloadSize)) } return cmd, headersAndPayload, nil } func (r *connectionReader) readAtLeast(size uint32) error { if r.buffer.WritableBytes() < size { // There's not enough room in the current buffer to read the requested amount of data totalFrameSize := r.buffer.ReadableBytes() + size if r.buffer.ReadableBytes()+size > r.buffer.Capacity() { // Resize to a bigger buffer to avoid continuous resizing r.buffer.Resize(totalFrameSize * 2) } else { // Compact the buffer by moving the partial data to the beginning. // This will have enough room for reading the remainder of the data r.buffer.MoveToFront() } } n, err := io.ReadAtLeast(r.cnx.cnx, r.buffer.WritableSlice(), int(size)) if err != nil { r.cnx.TriggerClose() return err } r.buffer.WrittenBytes(uint32(n)) return nil } func (r *connectionReader) deserializeCmd(data []byte) (*pb.BaseCommand, error) { cmd := &pb.BaseCommand{} err := proto.Unmarshal(data, cmd) if err != nil { r.cnx.log.WithError(err).Warn("Failed to parse protobuf command") r.cnx.TriggerClose() return nil, err } return cmd, nil }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "math/rand" "sync/atomic" "time" ) type defaultRouter struct { clock Clock shiftIdx uint32 maxBatchingDelay time.Duration hashFunc func(string) uint32 msgCounter uint32 } type Clock func() uint64 // NewSystemClock init system clock and return current time. func NewSystemClock() Clock { return func() uint64 { return uint64(time.Now().UnixNano()) } } // NewDefaultRouter set the message routing mode for the partitioned producer. // Default routing mode is round-robin routing. func NewDefaultRouter(clock Clock, hashFunc func(string) uint32, maxBatchingDelay time.Duration, disableBatching bool) func(string, uint32) int { state := &defaultRouter{ clock: clock, shiftIdx: rand.Uint32(), maxBatchingDelay: maxBatchingDelay, hashFunc: hashFunc, msgCounter: 0, } return func(key string, numPartitions uint32) int { if numPartitions == 1 { // When there are no partitions, don't even bother return 0 } if key != "" { // When a key is specified, use the hash of that key return int(state.hashFunc(key) % numPartitions) } // If there's no key, we do round-robin across partition, sticking with a given // partition for a certain amount of time, to ensure we can have a decent amount // of batching of the messages. // //currentMs / maxBatchingDelayMs + startPtnIdx if !disableBatching && maxBatchingDelay.Nanoseconds() > 0 { n := uint32(state.clock()/uint64(maxBatchingDelay.Nanoseconds())) + state.shiftIdx return int(n % numPartitions) } p := int(state.msgCounter % numPartitions) atomic.AddUint32(&state.msgCounter, 1) return p } }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 "github.com/spaolacci/murmur3" // JavaStringHash and Java String.hashCode() equivalent func JavaStringHash(s string) uint32 { var h uint32 for i, size := 0, len(s); i < size; i++ { h = 31*h + uint32(s[i]) } return h } // Murmur3_32Hash use Murmur3 hashing function func Murmur3_32Hash(s string) uint32 { h := murmur3.New32() _, err := h.Write([]byte(s)) if err != nil { return 0 } // Maintain compatibility with values used in Java client return h.Sum32() & 0x7fffffff }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "errors" "fmt" "net/url" "github.com/apache/pulsar-client-go/pulsar/internal/pb" "github.com/golang/protobuf/proto" log "github.com/sirupsen/logrus" ) // LookupResult encapsulates a struct for lookup a request, containing two parts: LogicalAddr, PhysicalAddr. type LookupResult struct { LogicalAddr *url.URL PhysicalAddr *url.URL } // LookupService is a interface of lookup service. type LookupService interface { // Lookup perform a lookup for the given topic, confirm the location of the broker // where the topic is located, and return the LookupResult. Lookup(topic string) (*LookupResult, error) } type lookupService struct { rpcClient RPCClient serviceURL *url.URL tlsEnabled bool } // NewLookupService init a lookup service struct and return an object of LookupService. func NewLookupService(rpcClient RPCClient, serviceURL *url.URL, tlsEnabled bool) LookupService { return &lookupService{ rpcClient: rpcClient, serviceURL: serviceURL, tlsEnabled: tlsEnabled, } } 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()) } if err != nil { return nil, nil, err } var physicalAddr *url.URL if lr.GetProxyThroughServiceUrl() { physicalAddr = ls.serviceURL } else { physicalAddr = logicalAddress } return logicalAddress, physicalAddr, nil } // Follow brokers redirect up to certain number of times const lookupResultMaxRedirect = 20 func (ls *lookupService) Lookup(topic string) (*LookupResult, error) { id := ls.rpcClient.NewRequestID() res, err := ls.rpcClient.RequestToAnyBroker(id, pb.BaseCommand_LOOKUP, &pb.CommandLookupTopic{ RequestId: &id, Topic: &topic, Authoritative: proto.Bool(false), }) if err != nil { return nil, err } log.Debugf("Got topic{%s} lookup response: %+v", topic, res) for i := 0; i < lookupResultMaxRedirect; i++ { lr := res.Response.LookupTopicResponse switch *lr.Response { case pb.CommandLookupTopicResponse_Redirect: logicalAddress, physicalAddr, err := ls.getBrokerAddress(lr) if err != nil { return nil, err } log.Debugf("Follow topic{%s} redirect to broker. %v / %v - Use proxy: %v", 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, }) if err != nil { return nil, err } // Process the response at the top of the loop continue case pb.CommandLookupTopicResponse_Connect: 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 case pb.CommandLookupTopicResponse_Failed: errorMsg := "" if lr.Error != nil { errorMsg = lr.Error.String() } log.Warnf("Failed to lookup topic: %s, error msg: %s", topic, errorMsg) return nil, fmt.Errorf("failed to lookup topic: %s", errorMsg) } } return nil, errors.New("exceeded max number of redirection during topic lookup") }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "errors" "net/url" "sync" "sync/atomic" "time" "github.com/apache/pulsar-client-go/pulsar/internal/pb" "github.com/golang/protobuf/proto" log "github.com/sirupsen/logrus" ) type RPCResult struct { Response *pb.BaseCommand Cnx Connection } type RPCClient interface { // Create a new unique request id NewRequestID() uint64 NewProducerID() uint64 NewConsumerID() uint64 // Send a request and block until the result is available RequestToAnyBroker(requestID uint64, cmdType pb.BaseCommand_Type, message proto.Message) (*RPCResult, error) Request(logicalAddr *url.URL, physicalAddr *url.URL, requestID uint64, cmdType pb.BaseCommand_Type, message proto.Message) (*RPCResult, error) RequestOnCnxNoWait(cnx Connection, cmdType pb.BaseCommand_Type, message proto.Message) RequestOnCnx(cnx Connection, requestID uint64, cmdType pb.BaseCommand_Type, message proto.Message) (*RPCResult, error) } type rpcClient struct { serviceURL *url.URL pool ConnectionPool requestTimeout time.Duration requestIDGenerator uint64 producerIDGenerator uint64 consumerIDGenerator uint64 log *log.Entry } func NewRPCClient(serviceURL *url.URL, pool ConnectionPool, requestTimeout time.Duration) RPCClient { return &rpcClient{ serviceURL: serviceURL, pool: pool, requestTimeout: requestTimeout, log: log.WithField("serviceURL", serviceURL), } } func (c *rpcClient) RequestToAnyBroker(requestID uint64, cmdType pb.BaseCommand_Type, message proto.Message) (*RPCResult, error) { return c.Request(c.serviceURL, c.serviceURL, requestID, cmdType, message) } func (c *rpcClient) Request(logicalAddr *url.URL, physicalAddr *url.URL, requestID uint64, cmdType pb.BaseCommand_Type, message proto.Message) (*RPCResult, error) { cnx, err := c.getConn(logicalAddr, physicalAddr) if err != nil { return nil, err } type Res struct { *RPCResult error } ch := make(chan Res, 10) // TODO: in here, the error of callback always nil cnx.SendRequest(requestID, baseCommand(cmdType, message), func(response *pb.BaseCommand, err error) { ch <- Res{&RPCResult{ Cnx: cnx, Response: response, }, err} close(ch) }) select { case res := <-ch: return res.RPCResult, res.error case <-time.After(c.requestTimeout): return nil, errors.New("request timed out") } } func (c *rpcClient) getConn(logicalAddr *url.URL, physicalAddr *url.URL) (Connection, error) { cnx, err := c.pool.GetConnection(logicalAddr, physicalAddr) backoff := Backoff{1 * time.Second} startTime := time.Now() var retryTime time.Duration if err != nil { for time.Since(startTime) < c.requestTimeout { retryTime = backoff.Next() c.log.Debugf("Reconnecting to broker in {%v} with timeout in {%v}", retryTime, c.requestTimeout) time.Sleep(retryTime) cnx, err = c.pool.GetConnection(logicalAddr, physicalAddr) if err == nil { c.log.Debugf("retry connection success") return cnx, nil } } return nil, err } return cnx, nil } func (c *rpcClient) RequestOnCnx(cnx Connection, requestID uint64, cmdType pb.BaseCommand_Type, message proto.Message) (*RPCResult, error) { wg := sync.WaitGroup{} wg.Add(1) rpcResult := &RPCResult{ Cnx: cnx, } var rpcErr error cnx.SendRequest(requestID, baseCommand(cmdType, message), func(response *pb.BaseCommand, err error) { rpcResult.Response = response rpcErr = err wg.Done() }) wg.Wait() return rpcResult, rpcErr } func (c *rpcClient) RequestOnCnxNoWait(cnx Connection, cmdType pb.BaseCommand_Type, message proto.Message) { cnx.SendRequestNoWait(baseCommand(cmdType, message)) } func (c *rpcClient) NewRequestID() uint64 { return atomic.AddUint64(&c.requestIDGenerator, 1) } func (c *rpcClient) NewProducerID() uint64 { return atomic.AddUint64(&c.producerIDGenerator, 1) } func (c *rpcClient) NewConsumerID() uint64 { return atomic.AddUint64(&c.consumerIDGenerator, 1) }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 // Semaphore is a channel of bool, used to receive a bool type semaphore. type Semaphore chan bool // Acquire a permit from this semaphore, blocking until one is available. // Acquire a permit, if one is available and returns immediately, // reducing the number of available permits by one. func (s Semaphore) Acquire() { s <- true } // Release a permit, returning it to the semaphore. // Release a permit, increasing the number of available permits by // one. If any threads are trying to acquire a permit, then one is // selected and given the permit that was just released. That thread // is (re)enabled for thread scheduling purposes. // There is no requirement that a thread that releases a permit must // have acquired that permit by calling Acquire(). // Correct usage of a semaphore is established by programming convention // in the application. func (s Semaphore) Release() { <-s }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "errors" "fmt" "strconv" "strings" ) // TopicName abstract a struct contained in a Topic type TopicName struct { Name string Namespace string Partition int } const ( publicTenant = "public" defaultNamespace = "default" partitionedTopicSuffix = "-partition-" ) // ParseTopicName parse the given topic name and return TopicName. func ParseTopicName(topic string) (*TopicName, error) { // The topic name can be in two different forms, one is fully qualified topic name, // the other one is short topic name if !strings.Contains(topic, "://") { // The short topic name can be: // - <topic> // - <tenant>/<namespace>/<topic> // - <tenant>/<cluster>/<namespace>/<topic> parts := strings.Split(topic, "/") if len(parts) == 3 || len(parts) == 4 { topic = "persistent://" + topic } else if len(parts) == 1 { topic = "persistent://" + publicTenant + "/" + defaultNamespace + "/" + parts[0] } else { return nil, errors.New( "Invalid short topic name '" + topic + "', it should be in the format of <tenant>/<namespace>/<topic> or <topic>") } } tn := &TopicName{} // The fully qualified topic name can be in two different forms: // new: persistent://tenant/namespace/topic // legacy: persistent://tenant/cluster/namespace/topic parts := strings.SplitN(topic, "://", 2) domain := parts[0] if domain != "persistent" && domain != "non-persistent" { return nil, errors.New("Invalid topic domain: " + domain) } rest := parts[1] var err error // The rest of the name can be in different forms: // new: tenant/namespace/<localName> // legacy: tenant/cluster/namespace/<localName> // Examples of localName: // 1. some/name/xyz// // 2. /xyz-123/feeder-2 parts = strings.SplitN(rest, "/", 4) if len(parts) == 3 { // New topic name without cluster name tn.Namespace = parts[0] + "/" + parts[1] } else if len(parts) == 4 { // Legacy topic name that includes cluster name tn.Namespace = fmt.Sprintf("%s/%s/%s", parts[0], parts[1], parts[2]) } else { return nil, errors.New("Invalid topic name: " + topic) } tn.Name = topic tn.Partition, err = getPartitionIndex(topic) if err != nil { return nil, err } return tn, nil } func TopicNameWithoutPartitionPart(tn *TopicName) string { if tn.Partition < 0 { return tn.Name } idx := strings.LastIndex(tn.Name, partitionedTopicSuffix) if idx > 0 { return tn.Name[:idx] } return tn.Name } func getPartitionIndex(topic string) (int, error) { if strings.Contains(topic, partitionedTopicSuffix) { idx := strings.LastIndex(topic, "-") + 1 return strconv.Atoi(topic[idx:]) } return -1, nil }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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 ( "strconv" "strings" "sync/atomic" "time" "github.com/pkg/errors" ) // TimestampMillis return a time unix nano. func TimestampMillis(t time.Time) uint64 { return uint64(t.UnixNano()) / uint64(time.Millisecond) } // GetAndAdd perform atomic read and update func GetAndAdd(n *uint64, diff uint64) uint64 { for { v := *n if atomic.CompareAndSwapUint64(n, v, v+diff) { return v } } } func ParseRelativeTimeInSeconds(relativeTime string) (time.Duration, error) { if relativeTime == "" { return -1, errors.New("time can not be empty") } unitTime := relativeTime[len(relativeTime)-1:] t := relativeTime[:len(relativeTime)-1] timeValue, err := strconv.ParseInt(t, 10, 64) if err != nil { return -1, errors.Errorf("invalid time '%s'", t) } switch strings.ToLower(unitTime) { case "s": return time.Duration(timeValue) * time.Second, nil case "m": return time.Duration(timeValue) * time.Minute, nil case "h": return time.Duration(timeValue) * time.Hour, nil case "d": return time.Duration(timeValue) * time.Hour * 24, nil case "w": return time.Duration(timeValue) * time.Hour * 24 * 7, nil case "y": return time.Duration(timeValue) * time.Hour * 24 * 7 * 365, nil default: return -1, errors.Errorf("invalid time unit '%s'", unitTime) } }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "time" ) // ProducerMessage abstraction used in Pulsar producer type ProducerMessage struct { // Payload for the message Payload []byte // Key sets the key of the message for routing policy Key string // Properties attach application defined properties on the message Properties map[string]string // EventTime set the event time for a given message // By default, messages don't have an event time associated, while the publish // time will be be always present. // Set the event time to a non-zero timestamp to explicitly declare the time // that the event "happened", as opposed to when the message is being published. EventTime time.Time // ReplicationClusters override the replication clusters for this message. ReplicationClusters []string // SequenceID set the sequence id to assign to the current message SequenceID *int64 // Request to deliver the message only after the specified relative delay. // Note: messages are only delivered with delay when a consumer is consuming // through a `SubscriptionType=Shared` subscription. With other subscription // types, the messages will still be delivered immediately. DeliverAfter time.Duration // Deliver the message only at or after the specified absolute timestamp. // Note: messages are only delivered with delay when a consumer is consuming // through a `SubscriptionType=Shared` subscription. With other subscription // types, the messages will still be delivered immediately. DeliverAt time.Time } // Message abstraction used in Pulsar type Message interface { // Topic get the topic from which this message originated from Topic() string // Properties are application defined key/value pairs that will be attached to the message. // Return the properties attached to the message. Properties() map[string]string // Payload get the payload of the message Payload() []byte // ID get the unique message ID associated with this message. // The message id can be used to univocally refer to a message without having the keep the entire payload in memory. ID() MessageID // PublishTime get the publish time of this message. The publish time is the timestamp that a client // publish the message. PublishTime() time.Time // EventTime get the event time associated with this message. It is typically set by the applications via // `ProducerMessage.EventTime`. // If EventTime is 0, it means there isn't any event time associated with this message. EventTime() time.Time // Key get the key of the message, if any Key() string // Get message redelivery count, redelivery count maintain in pulsar broker. When client nack acknowledge messages, // broker will dispatch message again with message redelivery count in CommandMessage defined. // // Message redelivery increases monotonically in a broker, when topic switch ownership to a another broker // redelivery count will be recalculated. RedeliveryCount() uint32 // Check whether the message is replicated from other cluster. IsReplicated() bool // Get name of cluster, from which the message is replicated. GetReplicatedFrom() string } // MessageID identifier for a particular message type MessageID interface { // Serialize the message id into a sequence of bytes that can be stored somewhere else Serialize() []byte } // DeserializeMessageID reconstruct a MessageID object from its serialized representation func DeserializeMessageID(data []byte) (MessageID, error) { return deserializeMessageID(data) } // EarliestMessageID returns a messageID that points to the earliest message available in a topic func EarliestMessageID() MessageID { return newMessageID(-1, -1, -1, -1) } // LatestMessage returns a messageID that points to the latest message func LatestMessageID() MessageID { return newMessageID(math.MaxInt64, math.MaxInt64, -1, -1) }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "time" log "github.com/sirupsen/logrus" ) type redeliveryConsumer interface { Redeliver(msgIds []messageID) } type negativeAcksTracker struct { sync.Mutex doneCh chan interface{} negativeAcks map[messageID]time.Time rc redeliveryConsumer tick *time.Ticker delay time.Duration } func newNegativeAcksTracker(rc redeliveryConsumer, delay time.Duration) *negativeAcksTracker { t := &negativeAcksTracker{ doneCh: make(chan interface{}), negativeAcks: make(map[messageID]time.Time), rc: rc, tick: time.NewTicker(delay / 3), delay: delay, } go t.track() return t } 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{ ledgerID: msgID.ledgerID, entryID: msgID.entryID, batchIdx: 0, } t.Lock() defer t.Unlock() _, present := t.negativeAcks[batchMsgID] if present { // The batch is already being tracked return } targetTime := time.Now().Add(t.delay) t.negativeAcks[batchMsgID] = targetTime } func (t *negativeAcksTracker) track() { for { select { case <-t.doneCh: log.Debug("Closing nack tracker") return case <-t.tick.C: { t.Lock() now := time.Now() msgIds := make([]messageID, 0) for msgID, targetTime := range t.negativeAcks { log.Debugf("MsgId: %v -- targetTime: %v -- now: %v", msgID, targetTime, now) if targetTime.Before(now) { log.Debugf("Adding MsgId: %v", msgID) msgIds = append(msgIds, msgID) delete(t.negativeAcks, msgID) } } t.Unlock() if len(msgIds) > 0 { t.rc.Redeliver(msgIds) } } } } } func (t *negativeAcksTracker) Close() { t.doneCh <- nil }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" log "github.com/sirupsen/logrus" "github.com/apache/pulsar-client-go/pulsar/internal" ) type producer struct { sync.Mutex client *client options *ProducerOptions topic string producers []Producer numPartitions uint32 messageRouter func(*ProducerMessage, TopicMetadata) int ticker *time.Ticker log *log.Entry } const defaultBatchingMaxPublishDelay = 10 * time.Millisecond var partitionsAutoDiscoveryInterval = 1 * time.Minute func getHashingFunction(s HashingScheme) func(string) uint32 { switch s { case JavaStringHash: return internal.JavaStringHash case Murmur3_32Hash: return internal.Murmur3_32Hash default: return internal.JavaStringHash } } func newProducer(client *client, options *ProducerOptions) (*producer, error) { if options.Topic == "" { return nil, newError(ResultInvalidTopicName, "Topic name is required for producer") } p := &producer{ options: options, topic: options.Topic, client: client, log: log.WithField("topic", options.Topic), } var batchingMaxPublishDelay time.Duration if options.BatchingMaxPublishDelay != 0 { batchingMaxPublishDelay = options.BatchingMaxPublishDelay } else { batchingMaxPublishDelay = defaultBatchingMaxPublishDelay } if options.MessageRouter == nil { internalRouter := internal.NewDefaultRouter( internal.NewSystemClock(), getHashingFunction(options.HashingScheme), batchingMaxPublishDelay, options.DisableBatching) p.messageRouter = func(message *ProducerMessage, metadata TopicMetadata) int { return internalRouter(message.Key, metadata.NumPartitions()) } } else { p.messageRouter = options.MessageRouter } err := p.internalCreatePartitionsProducers() if err != nil { return nil, err } p.ticker = time.NewTicker(partitionsAutoDiscoveryInterval) go func() { for range p.ticker.C { p.log.Debug("Auto discovering new partitions") p.internalCreatePartitionsProducers() } }() return p, nil } func (p *producer) internalCreatePartitionsProducers() error { partitions, err := p.client.TopicPartitions(p.topic) if err != nil { return err } oldNumPartitions := 0 newNumPartitions := len(partitions) p.Lock() defer p.Unlock() oldProducers := p.producers if oldProducers != nil { oldNumPartitions = len(oldProducers) if oldNumPartitions == newNumPartitions { p.log.Debug("Number of partitions in topic has not changed") return nil } p.log.WithField("old_partitions", oldNumPartitions). WithField("new_partitions", newNumPartitions). Info("Changed number of partitions in topic") } p.producers = make([]Producer, newNumPartitions) // Copy over the existing consumer instances for i := 0; i < oldNumPartitions; i++ { p.producers[i] = oldProducers[i] } type ProducerError struct { partition int prod Producer err error } partitionsToAdd := newNumPartitions - oldNumPartitions c := make(chan ProducerError, partitionsToAdd) for partitionIdx := oldNumPartitions; partitionIdx < newNumPartitions; partitionIdx++ { partition := partitions[partitionIdx] go func(partitionIdx int, partition string) { prod, e := newPartitionProducer(p.client, partition, p.options, partitionIdx) c <- ProducerError{ partition: partitionIdx, prod: prod, err: e, } }(partitionIdx, partition) } for i := 0; i < partitionsToAdd; i++ { pe, ok := <-c if ok { if pe.err != nil { err = pe.err } else { p.producers[pe.partition] = pe.prod } } } if err != nil { // Since there were some failures, cleanup all the partitions that succeeded in creating the producers for _, producer := range p.producers { if producer != nil { producer.Close() } } return err } atomic.StoreUint32(&p.numPartitions, uint32(len(p.producers))) return nil } func (p *producer) Topic() string { return p.topic } func (p *producer) Name() string { p.Lock() defer p.Unlock() return p.producers[0].Name() } func (p *producer) NumPartitions() uint32 { return atomic.LoadUint32(&p.numPartitions) } func (p *producer) Send(ctx context.Context, msg *ProducerMessage) (MessageID, error) { p.Lock() partition := p.messageRouter(msg, p) pp := p.producers[partition] p.Unlock() return pp.Send(ctx, msg) } func (p *producer) SendAsync(ctx context.Context, msg *ProducerMessage, callback func(MessageID, *ProducerMessage, error)) { p.Lock() partition := p.messageRouter(msg, p) pp := p.producers[partition] p.Unlock() pp.SendAsync(ctx, msg, callback) } func (p *producer) LastSequenceID() int64 { p.Lock() defer p.Unlock() var maxSeq int64 = -1 for _, pp := range p.producers { s := pp.LastSequenceID() if s > maxSeq { maxSeq = s } } return maxSeq } func (p *producer) Flush() error { p.Lock() defer p.Unlock() for _, pp := range p.producers { if err := pp.Flush(); err != nil { return err } } return nil } func (p *producer) Close() { p.Lock() defer p.Unlock() for _, pp := range p.producers { pp.Close() } p.client.handlers.Del(p) }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "errors" "sync" "sync/atomic" "time" "github.com/golang/protobuf/proto" log "github.com/sirupsen/logrus" "github.com/apache/pulsar-client-go/pulsar/internal" "github.com/apache/pulsar-client-go/pulsar/internal/pb" ) const ( // producer states producerInit int32 = iota producerReady producerClosing producerClosed ) var errFailAddBatch = errors.New("message send failed") type partitionProducer struct { state int32 client *client topic string log *log.Entry cnx internal.Connection options *ProducerOptions producerName string producerID uint64 batchBuilder *internal.BatchBuilder sequenceIDGenerator *uint64 batchFlushTicker *time.Ticker // Channel where app is posting messages to be published eventsChan chan interface{} publishSemaphore internal.Semaphore pendingQueue internal.BlockingQueue lastSequenceID int64 partitionIdx int } func newPartitionProducer(client *client, topic string, options *ProducerOptions, partitionIdx int) ( *partitionProducer, error) { var batchingMaxPublishDelay time.Duration if options.BatchingMaxPublishDelay != 0 { batchingMaxPublishDelay = options.BatchingMaxPublishDelay } else { batchingMaxPublishDelay = defaultBatchingMaxPublishDelay } var maxPendingMessages int if options.MaxPendingMessages == 0 { maxPendingMessages = 1000 } else { maxPendingMessages = options.MaxPendingMessages } p := &partitionProducer{ state: producerInit, log: log.WithField("topic", topic), client: client, topic: topic, options: options, producerID: client.rpcClient.NewProducerID(), eventsChan: make(chan interface{}, 10), batchFlushTicker: time.NewTicker(batchingMaxPublishDelay), publishSemaphore: make(internal.Semaphore, maxPendingMessages), pendingQueue: internal.NewBlockingQueue(maxPendingMessages), lastSequenceID: -1, partitionIdx: partitionIdx, } if options.Name != "" { p.producerName = options.Name } err := p.grabCnx() if err != nil { log.WithError(err).Errorf("Failed to create producer") return nil, err } p.log = p.log.WithField("producer_name", p.producerName) p.log.WithField("cnx", p.cnx.ID()).Info("Created producer") atomic.StoreInt32(&p.state, producerReady) go p.runEventsLoop() return p, nil } func (p *partitionProducer) grabCnx() error { lr, err := p.client.lookupService.Lookup(p.topic) 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() cmdProducer := &pb.CommandProducer{ RequestId: proto.Uint64(id), Topic: proto.String(p.topic), Encrypted: nil, ProducerId: proto.Uint64(p.producerID), Schema: nil, } if p.producerName != "" { cmdProducer.ProducerName = proto.String(p.producerName) } if len(p.options.Properties) > 0 { cmdProducer.Metadata = toKeyValues(p.options.Properties) } 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") return err } p.producerName = res.Response.ProducerSuccess.GetProducerName() if p.batchBuilder == nil { p.batchBuilder, err = internal.NewBatchBuilder(p.options.BatchingMaxMessages, p.producerName, p.producerID, pb.CompressionType(p.options.CompressionType)) if err != nil { return err } } if p.sequenceIDGenerator == nil { nextSequenceID := uint64(res.Response.ProducerSuccess.GetLastSequenceId() + 1) p.sequenceIDGenerator = &nextSequenceID } p.cnx = res.Cnx p.cnx.RegisterListener(p.producerID, p) p.log.WithField("cnx", res.Cnx.ID()).Debug("Connected producer") if p.pendingQueue.Size() > 0 { p.log.Infof("Resending %d pending batches", p.pendingQueue.Size()) for it := p.pendingQueue.Iterator(); it.HasNext(); { p.cnx.WriteData(it.Next().(*pendingItem).batchData) } } return nil } type connectionClosed struct{} func (p *partitionProducer) ConnectionClosed() { // Trigger reconnection in the produce goroutine p.log.WithField("cnx", p.cnx.ID()).Warn("Connection was closed") p.eventsChan <- &connectionClosed{} } func (p *partitionProducer) reconnectToBroker() { backoff := internal.Backoff{} for { if atomic.LoadInt32(&p.state) != producerReady { // Producer is already closing return } d := backoff.Next() p.log.Info("Reconnecting to broker in ", d) time.Sleep(d) err := p.grabCnx() if err == nil { // Successfully reconnected p.log.WithField("cnx", p.cnx.ID()).Info("Reconnected producer to broker") return } } } func (p *partitionProducer) runEventsLoop() { for { select { case i := <-p.eventsChan: switch v := i.(type) { case *sendRequest: p.internalSend(v) case *connectionClosed: p.reconnectToBroker() case *flushRequest: p.internalFlush(v) case *closeProducer: p.internalClose(v) return } case <-p.batchFlushTicker.C: p.internalFlushCurrentBatch() } } } func (p *partitionProducer) Topic() string { return p.topic } func (p *partitionProducer) Name() string { return p.producerName } func (p *partitionProducer) internalSend(request *sendRequest) { p.log.Debug("Received send request: ", *request) msg := request.msg deliverAt := msg.DeliverAt if msg.DeliverAfter.Nanoseconds() > 0 { deliverAt = time.Now().Add(msg.DeliverAfter) } sendAsBatch := !p.options.DisableBatching && msg.ReplicationClusters == nil && deliverAt.UnixNano() < 0 smm := &pb.SingleMessageMetadata{ PayloadSize: proto.Int(len(msg.Payload)), } if msg.EventTime.UnixNano() != 0 { smm.EventTime = proto.Uint64(internal.TimestampMillis(msg.EventTime)) } if msg.Key != "" { smm.PartitionKey = proto.String(msg.Key) } if msg.Properties != nil { smm.Properties = internal.ConvertFromStringMap(msg.Properties) } var sequenceID uint64 if msg.SequenceID != nil { sequenceID = uint64(*msg.SequenceID) } else { sequenceID = internal.GetAndAdd(p.sequenceIDGenerator, 1) } added := p.batchBuilder.Add(smm, sequenceID, msg.Payload, request, msg.ReplicationClusters, deliverAt) if !added { // 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, sequenceID, msg.Payload, request, msg.ReplicationClusters, deliverAt); !ok { p.publishSemaphore.Release() request.callback(nil, request.msg, errFailAddBatch) p.log.WithField("size", len(msg.Payload)). WithField("sequenceID", sequenceID). WithField("properties", msg.Properties). Error("unable to add message to batch") return } } if !sendAsBatch || request.flushImmediately { p.internalFlushCurrentBatch() } } type pendingItem struct { sync.Mutex batchData []byte sequenceID uint64 sendRequests []interface{} completed bool } func (p *partitionProducer) internalFlushCurrentBatch() { batchData, sequenceID, callbacks := p.batchBuilder.Flush() if batchData == nil { return } p.pendingQueue.Put(&pendingItem{ batchData: batchData, sequenceID: sequenceID, sendRequests: callbacks, }) p.cnx.WriteData(batchData) } func (p *partitionProducer) internalFlush(fr *flushRequest) { p.internalFlushCurrentBatch() pi, ok := p.pendingQueue.PeekLast().(*pendingItem) if !ok { fr.waitGroup.Done() return } // lock the pending request while adding requests // since the ReceivedSendReceipt func iterates over this list pi.Lock() defer pi.Unlock() if pi.completed { // 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 fr.waitGroup.Done() return } sendReq := &sendRequest{ msg: nil, callback: func(id MessageID, message *ProducerMessage, e error) { fr.err = e fr.waitGroup.Done() }, } pi.sendRequests = append(pi.sendRequests, sendReq) } func (p *partitionProducer) Send(ctx context.Context, msg *ProducerMessage) (MessageID, error) { wg := sync.WaitGroup{} wg.Add(1) var err error var msgID MessageID p.internalSendAsync(ctx, msg, func(ID MessageID, message *ProducerMessage, e error) { err = e msgID = ID wg.Done() }, true) wg.Wait() return msgID, err } func (p *partitionProducer) SendAsync(ctx context.Context, msg *ProducerMessage, callback func(MessageID, *ProducerMessage, error)) { p.publishSemaphore.Acquire() sr := &sendRequest{ ctx: ctx, msg: msg, callback: callback, flushImmediately: false, } p.eventsChan <- sr } func (p *partitionProducer) internalSendAsync(ctx context.Context, msg *ProducerMessage, callback func(MessageID, *ProducerMessage, error), flushImmediately bool) { p.publishSemaphore.Acquire() sr := &sendRequest{ ctx: ctx, msg: msg, callback: callback, flushImmediately: flushImmediately, } p.eventsChan <- sr } func (p *partitionProducer) ReceivedSendReceipt(response *pb.CommandSendReceipt) { pi, ok := p.pendingQueue.Peek().(*pendingItem) if !ok { p.log.Warnf("Received ack for %v although the pending queue is empty", response.GetMessageId()) return } if pi.sequenceID != response.GetSequenceId() { p.log.Warnf("Received ack for %v on sequenceId %v - expected: %v", response.GetMessageId(), response.GetSequenceId(), pi.sequenceID) return } // The ack was indeed for the expected item in the queue, we can remove it and trigger the callback p.pendingQueue.Poll() // lock the pending item while sending the requests pi.Lock() defer pi.Unlock() for idx, i := range pi.sendRequests { sr := i.(*sendRequest) if sr.msg != nil { atomic.StoreInt64(&p.lastSequenceID, int64(pi.sequenceID)) p.publishSemaphore.Release() } if sr.callback != nil { msgID := newMessageID( int64(response.MessageId.GetLedgerId()), int64(response.MessageId.GetEntryId()), idx, p.partitionIdx, ) sr.callback(msgID, sr.msg, nil) } } // Mark this pending item as done pi.completed = true } func (p *partitionProducer) internalClose(req *closeProducer) { defer req.waitGroup.Done() if !atomic.CompareAndSwapInt32(&p.state, producerReady, producerClosing) { return } p.log.Info("Closing producer") id := p.client.rpcClient.NewRequestID() _, err := p.client.rpcClient.RequestOnCnx(p.cnx, id, pb.BaseCommand_CLOSE_PRODUCER, &pb.CommandCloseProducer{ ProducerId: &p.producerID, RequestId: &id, }) if err != nil { p.log.WithError(err).Warn("Failed to close producer") } else { p.log.Info("Closed producer") } atomic.StoreInt32(&p.state, producerClosed) p.cnx.UnregisterListener(p.producerID) p.batchFlushTicker.Stop() } func (p *partitionProducer) LastSequenceID() int64 { return atomic.LoadInt64(&p.lastSequenceID) } func (p *partitionProducer) Flush() error { wg := sync.WaitGroup{} wg.Add(1) cp := &flushRequest{&wg, nil} p.eventsChan <- cp wg.Wait() return cp.err } func (p *partitionProducer) Close() { if atomic.LoadInt32(&p.state) != producerReady { // Producer is closing return } wg := sync.WaitGroup{} wg.Add(1) cp := &closeProducer{&wg} p.eventsChan <- cp wg.Wait() } type sendRequest struct { ctx context.Context msg *ProducerMessage callback func(MessageID, *ProducerMessage, error) flushImmediately bool } type closeProducer struct { waitGroup *sync.WaitGroup } type flushRequest struct { waitGroup *sync.WaitGroup err error }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" log "github.com/sirupsen/logrus" ) const ( defaultReceiverQueueSize = 1000 ) type reader struct { pc *partitionConsumer messageCh chan ConsumerMessage lastMessageInBroker *messageID log *log.Entry } func newReader(client *client, options ReaderOptions) (Reader, error) { if options.Topic == "" { return nil, newError(ResultInvalidConfiguration, "Topic is required") } if options.StartMessageID == nil { return nil, newError(ResultInvalidConfiguration, "StartMessageID is required") } subscriptionName := options.SubscriptionRolePrefix if subscriptionName == "" { subscriptionName = "reader" } subscriptionName += "-" + generateRandomName() receiverQueueSize := options.ReceiverQueueSize if receiverQueueSize == 0 { receiverQueueSize = defaultReceiverQueueSize } consumerOptions := &partitionConsumerOpts{ topic: options.Topic, consumerName: options.Name, subscription: subscriptionName, subscriptionType: Exclusive, receiverQueueSize: receiverQueueSize, startMessageID: options.StartMessageID.(*messageID), startMessageIDInclusive: options.StartMessageIDInclusive, subscriptionMode: nonDurable, readCompacted: options.ReadCompacted, metadata: options.Properties, nackRedeliveryDelay: defaultNackRedeliveryDelay, replicateSubscriptionState: false, } reader := &reader{ messageCh: make(chan ConsumerMessage), log: log.WithField("topic", options.Topic), } // Provide dummy dlq router with not dlq policy dlq, err := newDlqRouter(client, nil) if err != nil { return nil, err } pc, err := newPartitionConsumer(nil, client, consumerOptions, reader.messageCh, dlq) if err != nil { close(reader.messageCh) return nil, err } reader.pc = pc return reader, nil } func (r *reader) Topic() string { return r.pc.topic } func (r *reader) Next(ctx context.Context) (Message, error) { for { select { case cm, ok := <-r.messageCh: if !ok { return nil, ErrConsumerClosed } // 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().(*messageID) r.pc.lastDequeuedMsg = msgID r.pc.AckID(msgID) return cm.Message, nil case <-ctx.Done(): return nil, ctx.Err() } } } 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.greater(r.pc.lastDequeuedMsg) } if r.pc.options.startMessageIDInclusive { return r.lastMessageInBroker.greaterEqual(r.pc.startMessageID) } // Non-inclusive return r.lastMessageInBroker.greater(r.pc.startMessageID) } func (r *reader) Close() { r.pc.Close() }
// Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information // regarding copyright ownership. The ASF licenses this file // to you under the Apache License, Version 2.0 (the // "License"); you may not use this 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" "io/ioutil" "net/http" "path" "strings" "testing" "time" "github.com/apache/pulsar-client-go/pulsar/internal" pkgerrors "github.com/pkg/errors" ) const ( serviceURL = "pulsar://localhost:6650" serviceURLTLS = "pulsar+ssl://localhost:6651" webServiceURL = "http://localhost:8080" caCertsPath = "../integration-tests/certs/cacert.pem" tlsClientCertPath = "../integration-tests/certs/client-cert.pem" tlsClientKeyPath = "../integration-tests/certs/client-key.pem" tokenFilePath = "../integration-tests/tokens/token.txt" ) func newTopicName() string { return fmt.Sprintf("my-topic-%v", time.Now().Nanosecond()) } func newAuthTopicName() string { return fmt.Sprintf("private/auth/my-topic-%v", time.Now().Nanosecond()) } func testEndpoint(parts ...string) string { return webServiceURL + "/" + path.Join(parts...) } func jsonHeaders() http.Header { headers := http.Header{} headers.Add("Content-Type", "application/json") headers.Add("Accept", "application/json") return headers } func httpDelete(requestPaths ...string) error { var errs error for _, requestPath := range requestPaths { if err := httpDo(http.MethodDelete, requestPath, nil, nil); err != nil { errs = pkgerrors.Wrapf(err, "unable to delete url: %s"+requestPath) } } return errs } func httpPut(requestPath string, body interface{}) error { return httpDo(http.MethodPut, requestPath, body, nil) } func httpGet(requestPath string, out interface{}) error { return httpDo(http.MethodGet, requestPath, nil, out) } func httpDo(method string, requestPath string, in interface{}, out interface{}) error { client := http.DefaultClient endpoint := testEndpoint(requestPath) var body io.Reader inBytes, err := json.Marshal(in) if err != nil { return err } body = bytes.NewReader(inBytes) req, err := http.NewRequest(method, endpoint, body) if err != nil { return err } req.Header = jsonHeaders() resp, err := client.Do(req) if err != nil { return err } defer resp.Body.Close() if resp.StatusCode > 299 || resp.StatusCode < 200 { return fmt.Errorf("http error status code: %d", resp.StatusCode) } if out != nil { outBytes, err := ioutil.ReadAll(resp.Body) if err != nil { return err } return json.Unmarshal(outBytes, out) } return nil } func makeHTTPCall(t *testing.T, method string, url string, body string) { client := http.Client{} req, err := http.NewRequest(method, url, strings.NewReader(body)) if err != nil { t.Fatal(err) } req.Header.Set("Content-Type", "application/json") req.Header.Set("Accept", "application/json") res, err := client.Do(req) if err != nil { t.Fatal(err) } if res.Body != nil { _ = res.Body.Close() } } func createNamespace(namespace string, policy map[string]interface{}) error { return httpPut("admin/v2/namespaces/"+namespace, policy) } func createTopic(topic string) error { return httpPut("admin/v2/persistent/"+topic, nil) } func deleteTopic(topic string) error { return httpDelete("admin/v2/persistent/" + fmt.Sprintf("%s?force=true", topic)) } func topicStats(topic string) (map[string]interface{}, error) { var metadata map[string]interface{} err := httpGet("admin/v2/persistent/"+topicPath(topic)+"/stats", &metadata) return metadata, err } func topicPath(topic string) string { tn, _ := internal.ParseTopicName(topic) idx := strings.LastIndex(tn.Name, "/") if idx > 0 { return tn.Namespace + "/" + tn.Name[idx:] } return tn.Name }