You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/09/03 15:53:49 UTC

[GitHub] [pulsar-client-go] cckellogg commented on a change in pull request #610: Add reconsumer level later logic

cckellogg commented on a change in pull request #610:
URL: https://github.com/apache/pulsar-client-go/pull/610#discussion_r701975269



##########
File path: pulsar/consumer.go
##########
@@ -117,6 +117,9 @@ type ConsumerOptions struct {
 	// Default is false
 	RetryEnable bool
 
+	// Configuration for Retry delay level policy
+	DelayLevelUtil DelayLevelUtil

Review comment:
       What is a delay level util? Why does a util need to be exposed as a public API?

##########
File path: pulsar/consumer_partition.go
##########
@@ -271,6 +275,113 @@ func (pc *partitionConsumer) internalGetLastMessageID(req *getLastMsgIDRequest)
 	req.msgID, req.err = pc.requestGetLastMessageID()
 }
 
+func (pc *partitionConsumer) internalBeforeReconsume(msg Message, reconsumeOptions ReconsumeOptions) (Producer,
+	*ProducerMessage, string, error) {
+	if pc.getConsumerState() != consumerReady {
+		return nil, nil, "", errors.New("the Consumer already closed")
+	}
+	if reconsumeOptions == nil {
+		reconsumeOptions = NewReconsumeOptions()
+	}
+	if pc.options.delayLevelUtil == nil {
+		pc.options.delayLevelUtil = NewDelayLevelUtil(DefaultMessageDelayLevel)
+	}
+	propertiesMap := make(map[string]string)
+	if msg.Properties() != nil {
+		for k, v := range msg.Properties() {
+			propertiesMap[k] = v
+		}
+	}
+	reconsumeTimes := uint32(1)
+	delayLevels := -1
+	delayTime := reconsumeOptions.DelayTime()
+	if reconsumeOptions.DelayLevel() == -2 {
+		if v, ok := propertiesMap["DELAY"]; ok {
+			delayLevels, _ = strconv.Atoi(v)
+			if delayLevels == -1 {
+				delayLevels = 1
+			} else {
+				delayLevels++
+			}
+		} else {
+			delayLevels = 1
+		}
+		delayTime = pc.options.delayLevelUtil.GetDelayTime(delayLevels)
+	}
+	if reconsumeOptions.DelayLevel() >= 0 {
+		delayLevels = reconsumeOptions.DelayLevel()
+		delayTime = pc.options.delayLevelUtil.GetDelayTime(delayLevels)
+	}
+	if delayLevels > pc.options.delayLevelUtil.GetMaxDelayLevel() {
+		delayLevels = pc.options.delayLevelUtil.GetMaxDelayLevel()
+	}
+
+	if v, ok := propertiesMap["RECONSUMETIMES"]; ok {
+		reconsumeTimesUint64, err := strconv.ParseUint(v, 10, 64)
+		if err != nil {
+			return nil, nil, "", err
+		}
+		reconsumeTimes = uint32(reconsumeTimesUint64) + 1
+		propertiesMap["RECONSUMETIMES"] = strconv.FormatUint(uint64(reconsumeTimes), 10)

Review comment:
       Maybe add constants for these keys?

##########
File path: pulsar/consumer_impl.go
##########
@@ -246,6 +251,57 @@ func (c *consumer) runBackgroundPartitionDiscovery(period time.Duration) (cancel
 	}
 }
 
+func (c *consumer) ReconsumeLaterLevel(message Message, reconsumeOptions ReconsumeOptions) error {
+	if !c.options.RetryEnable {
+		return errors.New("the Consumer config retry disabled")
+	}
+	topicName, err := internal.ParseTopicName(message.Topic())
+	if err != nil {
+		return err
+	}
+	index := 0
+	if topicName.Partition >= 0 {
+		index = topicName.Partition
+	}
+	wg := sync.WaitGroup{}
+	wg.Add(1)
+	prod, prodMsg, desType, err := c.consumers[index].internalBeforeReconsume(message, reconsumeOptions)

Review comment:
       create a consumer variable and or pass it into the internal functions?

##########
File path: pulsar/reconsume_later_util.go
##########
@@ -0,0 +1,145 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package pulsar
+
+import (
+	"strconv"
+	"strings"
+	"time"
+
+	log "github.com/sirupsen/logrus"
+)
+
+const DefaultMessageDelayLevel = "1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h 2h"

Review comment:
       should this be private? why is this a string?

##########
File path: pulsar/consumer.go
##########
@@ -181,6 +184,14 @@ type Consumer interface {
 	// ReconsumeLater mark a message for redelivery after custom delay
 	ReconsumeLater(msg Message, delay time.Duration)
 
+	// Reconsume a message
+	ReconsumeLaterLevel(message Message, options ReconsumeOptions) error
+
+	// Reconsume a message Async
+	ReconsumeLaterLevelAsync(message Message,
+		options ReconsumeOptions,
+		callback func(MessageID, *ProducerMessage, error))

Review comment:
       Why is there a ProducerMessage in the callback?

##########
File path: pulsar/consumer_partition.go
##########
@@ -271,6 +275,113 @@ func (pc *partitionConsumer) internalGetLastMessageID(req *getLastMsgIDRequest)
 	req.msgID, req.err = pc.requestGetLastMessageID()
 }
 
+func (pc *partitionConsumer) internalBeforeReconsume(msg Message, reconsumeOptions ReconsumeOptions) (Producer,
+	*ProducerMessage, string, error) {
+	if pc.getConsumerState() != consumerReady {
+		return nil, nil, "", errors.New("the Consumer already closed")
+	}
+	if reconsumeOptions == nil {
+		reconsumeOptions = NewReconsumeOptions()
+	}
+	if pc.options.delayLevelUtil == nil {
+		pc.options.delayLevelUtil = NewDelayLevelUtil(DefaultMessageDelayLevel)
+	}
+	propertiesMap := make(map[string]string)
+	if msg.Properties() != nil {
+		for k, v := range msg.Properties() {
+			propertiesMap[k] = v
+		}
+	}
+	reconsumeTimes := uint32(1)
+	delayLevels := -1
+	delayTime := reconsumeOptions.DelayTime()
+	if reconsumeOptions.DelayLevel() == -2 {
+		if v, ok := propertiesMap["DELAY"]; ok {
+			delayLevels, _ = strconv.Atoi(v)
+			if delayLevels == -1 {
+				delayLevels = 1
+			} else {
+				delayLevels++
+			}
+		} else {
+			delayLevels = 1
+		}
+		delayTime = pc.options.delayLevelUtil.GetDelayTime(delayLevels)
+	}
+	if reconsumeOptions.DelayLevel() >= 0 {
+		delayLevels = reconsumeOptions.DelayLevel()
+		delayTime = pc.options.delayLevelUtil.GetDelayTime(delayLevels)
+	}
+	if delayLevels > pc.options.delayLevelUtil.GetMaxDelayLevel() {
+		delayLevels = pc.options.delayLevelUtil.GetMaxDelayLevel()
+	}
+
+	if v, ok := propertiesMap["RECONSUMETIMES"]; ok {
+		reconsumeTimesUint64, err := strconv.ParseUint(v, 10, 64)
+		if err != nil {
+			return nil, nil, "", err
+		}
+		reconsumeTimes = uint32(reconsumeTimesUint64) + 1
+		propertiesMap["RECONSUMETIMES"] = strconv.FormatUint(uint64(reconsumeTimes), 10)
+	} else {
+		propertiesMap["RECONSUMETIMES"] = "1"
+	}
+	propertiesMap["DELAY_TIME"] = fmt.Sprint(delayTime * (reconsumeOptions.DelayTimeUnit().Nanoseconds() / 1e6))
+	propertiesMap["DELAY"] = fmt.Sprint(int64(delayLevels) * (reconsumeOptions.DelayTimeUnit().Nanoseconds() / 1e6))
+
+	if reconsumeTimes == 1 {
+		propertiesMap["REAL_TOPIC"] = msg.Topic()
+		propertiesMap["RETRY_TOPIC"] = pc.dlq.policy.RetryLetterTopic
+		propertiesMap["ORIGIN_MESSAGE_ID"] = fmt.Sprint(msg.ID())
+		propertiesMap["producer_name"] = msg.ProducerName()
+	}
+
+	producerMsg := &ProducerMessage{
+		Payload:             msg.Payload(),
+		Properties:          propertiesMap,
+		ReplicationClusters: msg.(*message).replicationClusters,
+	}
+	if delayTime > 0 {
+		producerMsg.DeliverAfter = time.Duration(delayTime) * reconsumeOptions.DelayTimeUnit()
+	}
+	if msg.Key() != "" {
+		producerMsg.Key = msg.Key()
+	}
+	prod := pc.dlq.producer
+	desType := "retry"
+	if reconsumeTimes > pc.dlq.policy.MaxDeliveries {
+		propertiesMap["REAL_TOPIC"] = pc.dlq.policy.DeadLetterTopic
+		prod = pc.dlq.getProducer() // Get dead topic producer
+		desType = "dead"
+	}
+	// Default use the retry letter topic producer
+	if prod == nil {

Review comment:
       Should this check happen earlier? 

##########
File path: pulsar/consumer_impl.go
##########
@@ -246,6 +251,57 @@ func (c *consumer) runBackgroundPartitionDiscovery(period time.Duration) (cancel
 	}
 }
 
+func (c *consumer) ReconsumeLaterLevel(message Message, reconsumeOptions ReconsumeOptions) error {
+	if !c.options.RetryEnable {
+		return errors.New("the Consumer config retry disabled")
+	}
+	topicName, err := internal.ParseTopicName(message.Topic())
+	if err != nil {
+		return err
+	}
+	index := 0
+	if topicName.Partition >= 0 {
+		index = topicName.Partition
+	}
+	wg := sync.WaitGroup{}
+	wg.Add(1)
+	prod, prodMsg, desType, err := c.consumers[index].internalBeforeReconsume(message, reconsumeOptions)
+	if err != nil {
+		return err
+	}
+
+	c.consumers[index].internalReconsumeAsync(prod, message, prodMsg, desType,
+		func(id MessageID, producerMessage *ProducerMessage, e error) {
+			err = e
+			wg.Done()
+		})
+	wg.Wait()

Review comment:
       How long could this block for?

##########
File path: pulsar/reconsume_later_util.go
##########
@@ -0,0 +1,145 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package pulsar
+
+import (
+	"strconv"
+	"strings"
+	"time"
+
+	log "github.com/sirupsen/logrus"
+)
+
+const DefaultMessageDelayLevel = "1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h 2h"
+
+type DelayLevelUtil interface {
+	GetMaxDelayLevel() int
+	GetDelayTime(level int) int64
+	ParseDelayLevel() bool
+}
+
+type delayLevelUtil struct {
+	maxDelayLevel   int
+	levelString     string
+	delayLevelTable map[int]int64
+}
+
+func NewDelayLevelUtil(levelStr string) DelayLevelUtil {
+	delayLevelUtil := &delayLevelUtil{
+		levelString:     levelStr,
+		delayLevelTable: make(map[int]int64),
+	}
+	delayLevelUtil.ParseDelayLevel()
+	return delayLevelUtil
+}
+
+func (d *delayLevelUtil) GetMaxDelayLevel() int {
+	return d.maxDelayLevel
+}
+
+func (d *delayLevelUtil) GetDelayTime(level int) int64 {
+	if d.delayLevelTable == nil {
+		return 0
+	} else if level < 1 {
+		return 0
+	} else if level > d.maxDelayLevel {
+		return d.delayLevelTable[d.maxDelayLevel]
+	} else {
+		return d.delayLevelTable[level]
+	}
+}
+
+func (d *delayLevelUtil) ParseDelayLevel() bool {

Review comment:
       How does this leveling work?

##########
File path: pulsar/consumer_multitopic.go
##########
@@ -155,6 +156,46 @@ func (c *multiTopicConsumer) ReconsumeLater(msg Message, delay time.Duration) {
 	consumer.ReconsumeLater(msg, delay)
 }
 
+func (c *multiTopicConsumer) ReconsumeLaterLevel(message Message, reconsumeOptions ReconsumeOptions) error {
+	if !c.options.RetryEnable {
+		return errors.New("[ReconsumeLater]This Consumer config retry disabled. ")
+	}
+	topicName, err := internal.ParseTopicName(message.Topic())
+	if err != nil {
+		return err
+	}
+	topicNameWithoutPartition := internal.TopicNameWithoutPartitionPart(topicName)
+	for topic, consumer := range c.consumers {
+		consumerTopicName, _ := internal.ParseTopicName(topic)
+		if consumerTopicName.Name == topicNameWithoutPartition {
+			return consumer.ReconsumeLaterLevel(message, reconsumeOptions)
+		}
+	}
+	return errors.New("[ReconsumeLater]Topic not in multi topic consumer list. ")
+}
+
+func (c *multiTopicConsumer) ReconsumeLaterLevelAsync(message Message, reconsumeOptions ReconsumeOptions,
+	callback func(MessageID, *ProducerMessage, error)) {
+	if !c.options.RetryEnable {
+		c.log.Warn(errors.New("[ReconsumeLaterAsync]This Consumer config retry disabled. "))
+		return

Review comment:
       Do callbacks need to happen before returning?

##########
File path: pulsar/consumer_multitopic.go
##########
@@ -155,6 +156,46 @@ func (c *multiTopicConsumer) ReconsumeLater(msg Message, delay time.Duration) {
 	consumer.ReconsumeLater(msg, delay)
 }
 
+func (c *multiTopicConsumer) ReconsumeLaterLevel(message Message, reconsumeOptions ReconsumeOptions) error {
+	if !c.options.RetryEnable {
+		return errors.New("[ReconsumeLater]This Consumer config retry disabled. ")

Review comment:
       Let's remove [ReconsumeLater*] from logs and errors it's inconsistent with the rest of the code. Instead of `[ReconsumeLater]This Consumer config retry disabled. `  maybe `retries disabled for consumer unable to reconsume later`. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org