Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Issue 183] [Feat] Support cumulative acknowledge #903

Merged
merged 5 commits into from
Jan 3, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
feat: support cumulative ack
  • Loading branch information
Gleiphir2769 committed Dec 7, 2022
commit 3c2a6e6131e634afedd7ed9f60b82b89f9f171aa
8 changes: 8 additions & 0 deletions pulsar/consumer.go
Original file line number Diff line number Diff line change
Expand Up @@ -231,6 +231,14 @@ type Consumer interface {
// AckID the consumption of a single message, identified by its MessageID
AckID(MessageID) error

// CumulativeAck the reception of all the messages in the stream up to (and including)
// the provided message.
CumulativeAck(msg Message) error

// CumulativeAckID the reception of all the messages in the stream up to (and including)
// the provided message, identified by its MessageID
CumulativeAckID(msgID MessageID) error
BewareMyPower marked this conversation as resolved.
Show resolved Hide resolved

// ReconsumeLater mark a message for redelivery after custom delay
ReconsumeLater(msg Message, delay time.Duration)

Expand Down
22 changes: 22 additions & 0 deletions pulsar/consumer_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,8 @@ type acker interface {
// AckID does not handle errors returned by the Broker side, so no need to wait for doneCh to finish.
AckID(id MessageID) error
AckIDWithResponse(id MessageID) error
CumulativeAckID(msgID MessageID) error
CumulativeAckIDWithResponse(msgID MessageID) error
NackID(id MessageID)
NackMsg(msg Message)
}
Expand Down Expand Up @@ -477,6 +479,26 @@ func (c *consumer) AckID(msgID MessageID) error {
return c.consumers[msgID.PartitionIdx()].AckID(msgID)
}

// CumulativeAck the reception of all the messages in the stream up to (and including)
// the provided message, identified by its MessageID
func (c *consumer) CumulativeAck(msg Message) error {
return c.CumulativeAckID(msg.ID())
}

// CumulativeAckID the reception of all the messages in the stream up to (and including)
// the provided message, identified by its MessageID
func (c *consumer) CumulativeAckID(msgID MessageID) error {
if err := c.checkMsgIDPartition(msgID); err != nil {
return err
}

if c.options.AckWithResponse {
return c.consumers[msgID.PartitionIdx()].CumulativeAckIDWithResponse(msgID)
}

return c.consumers[msgID.PartitionIdx()].CumulativeAckID(msgID)
}

// ReconsumeLater mark a message for redelivery after custom delay
func (c *consumer) ReconsumeLater(msg Message, delay time.Duration) {
if delay < 0 {
Expand Down
27 changes: 27 additions & 0 deletions pulsar/consumer_multitopic.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,33 @@ func (c *multiTopicConsumer) AckID(msgID MessageID) error {
return mid.consumer.AckID(msgID)
}

// CumulativeAck the reception of all the messages in the stream up to (and including)
// the provided message
func (c *multiTopicConsumer) CumulativeAck(msg Message) error {
return c.CumulativeAckID(msg.ID())
}

// CumulativeAckID the reception of all the messages in the stream up to (and including)
// the provided message, identified by its MessageID
func (c *multiTopicConsumer) CumulativeAckID(msgID MessageID) error {
mid, ok := toTrackingMessageID(msgID)
if !ok {
c.log.Warnf("invalid message id type %T", msgID)
return errors.New("invalid message id type in multi_consumer")
}

if mid.consumer == nil {
c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID)
return errors.New("unable to ack message because consumer is nil")
}

if c.options.AckWithResponse {
return mid.consumer.CumulativeAckIDWithResponse(msgID)
}

return mid.consumer.CumulativeAckID(msgID)
}

func (c *multiTopicConsumer) ReconsumeLater(msg Message, delay time.Duration) {
names, err := validateTopicNames(msg.Topic())
if err != nil {
Expand Down
88 changes: 84 additions & 4 deletions pulsar/consumer_partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -431,6 +431,7 @@ func (pc *partitionConsumer) AckIDWithResponse(msgID MessageID) error {

ackReq := new(ackRequest)
ackReq.doneCh = make(chan struct{})
ackReq.ackType = individualAck
if !trackingID.Undefined() && trackingID.ack() {
pc.metrics.AcksCounter.Inc()
pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9)
Expand Down Expand Up @@ -463,6 +464,7 @@ func (pc *partitionConsumer) AckID(msgID MessageID) error {

ackReq := new(ackRequest)
ackReq.doneCh = make(chan struct{})
ackReq.ackType = individualAck
if !trackingID.Undefined() && trackingID.ack() {
pc.metrics.AcksCounter.Inc()
pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9)
Expand All @@ -477,6 +479,72 @@ func (pc *partitionConsumer) AckID(msgID MessageID) error {
return nil
}

func (pc *partitionConsumer) CumulativeAckIDWithResponse(msgID MessageID) error {
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer")
return errors.New("consumer state is closed")
}

trackingID, ok := toTrackingMessageID(msgID)
if !ok {
return errors.New("failed to convert trackingMessageID")
}

ackReq := new(ackRequest)
ackReq.doneCh = make(chan struct{})
ackReq.ackType = cumulativeAck
if !trackingID.Undefined() && trackingID.ack() {
pc.metrics.AcksCounter.Inc()
pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9)
ackReq.msgID = trackingID
// send ack request to eventsCh
pc.eventsCh <- ackReq
// wait for the request to complete
<-ackReq.doneCh

pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID)
}

if cmid, ok := toChunkedMessageID(msgID); ok {
pc.unAckChunksTracker.remove(cmid)
}

return nil
}

func (pc *partitionConsumer) CumulativeAckID(msgID MessageID) error {
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer")
return errors.New("consumer state is closed")
}

// chunk message id will be converted to tracking message id
trackingID, ok := toTrackingMessageID(msgID)
if !ok {
return errors.New("failed to convert trackingMessageID")
}

ackReq := new(ackRequest)
ackReq.doneCh = make(chan struct{})
ackReq.ackType = cumulativeAck
if !trackingID.Undefined() && trackingID.ack() {
pc.metrics.AcksCounter.Inc()
pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9)
ackReq.msgID = trackingID
// send ack request to eventsCh
pc.eventsCh <- ackReq
// No need to wait for ackReq.doneCh to finish

pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID)
}

if cmid, ok := toChunkedMessageID(msgID); ok {
pc.unAckChunksTracker.remove(cmid)
}

return nil
}

func (pc *partitionConsumer) NackID(msgID MessageID) {
if cmid, ok := toChunkedMessageID(msgID); ok {
pc.unAckChunksTracker.nack(cmid)
Expand Down Expand Up @@ -691,7 +759,13 @@ func (pc *partitionConsumer) internalAck(req *ackRequest) {
cmdAck := &pb.CommandAck{
ConsumerId: proto.Uint64(pc.consumerID),
MessageId: messageIDs,
AckType: pb.CommandAck_Individual.Enum(),
}

switch req.ackType {
case individualAck:
cmdAck.AckType = pb.CommandAck_Individual.Enum()
case cumulativeAck:
cmdAck.AckType = pb.CommandAck_Cumulative.Enum()
}

if pc.options.ackWithResponse {
Expand Down Expand Up @@ -1169,10 +1243,16 @@ func (pc *partitionConsumer) dispatcher() {
}
}

const (
individualAck = iota
cumulativeAck
)

type ackRequest struct {
doneCh chan struct{}
msgID trackingMessageID
err error
doneCh chan struct{}
msgID trackingMessageID
ackType int
err error
}

type unsubscribeRequest struct {
Expand Down
27 changes: 27 additions & 0 deletions pulsar/consumer_regex.go
Original file line number Diff line number Diff line change
Expand Up @@ -187,6 +187,33 @@ func (c *regexConsumer) AckID(msgID MessageID) error {
return mid.consumer.AckID(msgID)
}

// CumulativeAck the reception of all the messages in the stream up to (and including)
// the provided message.
func (c *regexConsumer) CumulativeAck(msg Message) error {
return c.CumulativeAckID(msg.ID())
}

// CumulativeAckID the reception of all the messages in the stream up to (and including)
// the provided message, identified by its MessageID
func (c *regexConsumer) CumulativeAckID(msgID MessageID) error {
mid, ok := toTrackingMessageID(msgID)
if !ok {
c.log.Warnf("invalid message id type %T", msgID)
return errors.New("invalid message id type")
}

if mid.consumer == nil {
c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID)
return errors.New("unable to ack message because consumer is nil")
}

if c.options.AckWithResponse {
return mid.consumer.CumulativeAckIDWithResponse(msgID)
}

return mid.consumer.CumulativeAckID(msgID)
}

func (c *regexConsumer) Nack(msg Message) {
if c.options.EnableDefaultNackBackoffPolicy || c.options.NackBackoffPolicy != nil {
msgID := msg.ID()
Expand Down
66 changes: 66 additions & 0 deletions pulsar/consumer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -731,6 +731,72 @@ func TestConsumerAck(t *testing.T) {
}
}

func TestConsumerCumulativeAck(t *testing.T) {
client, err := NewClient(ClientOptions{
URL: lookupURL,
})

assert.Nil(t, err)
defer client.Close()

topicName := newTopicName()
ctx := context.Background()

producer, err := client.CreateProducer(ProducerOptions{
Topic: topicName,
})
assert.Nil(t, err)
defer producer.Close()

consumer, err := client.Subscribe(ConsumerOptions{
Topic: topicName,
SubscriptionName: "sub-1",
Type: Exclusive,
})
assert.Nil(t, err)

const N = 100

for i := 0; i < N; i++ {
if _, err := producer.Send(ctx, &ProducerMessage{
Payload: []byte(fmt.Sprintf("msg-content-%d", i)),
}); err != nil {
t.Fatal(err)
}
}

for i := 0; i < N; i++ {
msg, err := consumer.Receive(ctx)
assert.Nil(t, err)
assert.Equal(t, fmt.Sprintf("msg-content-%d", i), string(msg.Payload()))

if i == N/2-1 {
// cumulative acks the first half of messages
consumer.CumulativeAck(msg)
}
}

consumer.Close()

// Subscribe again
consumer, err = client.Subscribe(ConsumerOptions{
Topic: topicName,
SubscriptionName: "sub-1",
Type: Exclusive,
})
assert.Nil(t, err)
defer consumer.Close()

// We should only receive the 2nd half of messages
for i := N / 2; i < N; i++ {
msg, err := consumer.Receive(ctx)
assert.Nil(t, err)
assert.Equal(t, fmt.Sprintf("msg-content-%d", i), string(msg.Payload()))

consumer.Ack(msg)
}
}

func TestConsumerNack(t *testing.T) {
client, err := NewClient(ClientOptions{
URL: lookupURL,
Expand Down
8 changes: 8 additions & 0 deletions pulsar/internal/pulsartracing/consumer_interceptor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,14 @@ func (c *mockConsumer) AckID(msgID pulsar.MessageID) error {
return nil
}

func (c *mockConsumer) CumulativeAck(msg pulsar.Message) error {
return nil
}

func (c *mockConsumer) CumulativeAckID(msgID pulsar.MessageID) error {
return nil
}

func (c *mockConsumer) ReconsumeLater(msg pulsar.Message, delay time.Duration) {}

func (c *mockConsumer) Nack(msg pulsar.Message) {}
Expand Down