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

Support ack response for Go SDK #776

Merged
merged 2 commits into from
May 24, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
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
7 changes: 7 additions & 0 deletions pulsar/consumer.go
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,13 @@ type ConsumerOptions struct {
// > Notice: the NackBackoffPolicy will not work with `consumer.NackID(MessageID)`
// > because we are not able to get the redeliveryCount from the message ID.
NackBackoffPolicy NackBackoffPolicy

// AckWithResponse is a return value added to Ack Command, and its purpose is to confirm whether Ack Command
// is executed correctly on the Broker side. When set to true, the error information returned by the Ack
// method contains the return value of the Ack Command processed by the Broker side; when set to false, the
// error information of the Ack method only contains errors that may occur in the Go SDK's own processing.
// Default: false
AckWithResponse bool
}

// Consumer is an interface that abstracts behavior of Pulsar's consumer
Expand Down
1 change: 1 addition & 0 deletions pulsar/consumer_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -361,6 +361,7 @@ func (c *consumer) internalTopicSubscribeToPartitions() error {
keySharedPolicy: c.options.KeySharedPolicy,
schema: c.options.Schema,
decryption: c.options.Decryption,
ackWithResponse: c.options.AckWithResponse,
}
cons, err := newPartitionConsumer(c, c.client, opts, c.messageCh, c.dlq, c.metrics)
ch <- ConsumerError{
Expand Down
11 changes: 11 additions & 0 deletions pulsar/consumer_partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ type partitionConsumerOpts struct {
keySharedPolicy *KeySharedPolicy
schema Schema
decryption *MessageDecryptionInfo
ackWithResponse bool
}

type partitionConsumer struct {
Expand Down Expand Up @@ -525,12 +526,22 @@ func (pc *partitionConsumer) internalAck(req *ackRequest) {
EntryId: proto.Uint64(uint64(msgID.entryID)),
}

reqID := pc.client.rpcClient.NewRequestID()
cmdAck := &pb.CommandAck{
ConsumerId: proto.Uint64(pc.consumerID),
MessageId: messageIDs,
AckType: pb.CommandAck_Individual.Enum(),
}

if pc.options.ackWithResponse {
_, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), reqID, pb.BaseCommand_ACK, cmdAck)
if err != nil {
pc.log.WithError(err).Error("Ack with response error")

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should handle the callback?

req.err = err
}
return
}

err := pc.client.rpcClient.RequestOnCnxNoWait(pc._getConn(), pb.BaseCommand_ACK, cmdAck)
if err != nil {
pc.log.Error("Connection was closed when request ack cmd")
Expand Down
35 changes: 35 additions & 0 deletions pulsar/consumer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1321,6 +1321,41 @@ func TestRLQ(t *testing.T) {
assert.Nil(t, checkMsg)
}

func TestAckWithResponse(t *testing.T) {
now := time.Now().Unix()
topic01 := fmt.Sprintf("persistent://public/default/topic-%d-01", now)
ctx := context.Background()

client, err := NewClient(ClientOptions{URL: lookupURL})
assert.Nil(t, err)
defer client.Close()

consumer, err := client.Subscribe(ConsumerOptions{
Topic: topic01,
SubscriptionName: "my-sub",
Type: Shared,
SubscriptionInitialPosition: SubscriptionPositionEarliest,
AckWithResponse: true,
})
assert.Nil(t, err)
defer consumer.Close()

producer01, err := client.CreateProducer(ProducerOptions{Topic: topic01})
assert.Nil(t, err)
defer producer01.Close()
for i := 0; i < 10; i++ {
_, err = producer01.Send(ctx, &ProducerMessage{Payload: []byte(fmt.Sprintf("MSG_01_%d", i))})
assert.Nil(t, err)
}

for i := 0; i < 10; i++ {
msg, err := consumer.Receive(ctx)
assert.Nil(t, err)
err = consumer.Ack(msg)
assert.Nil(t, err)
}
}

func TestRLQMultiTopics(t *testing.T) {
now := time.Now().Unix()
topic01 := fmt.Sprintf("persistent://public/default/topic-%d-1", now)
Expand Down
23 changes: 23 additions & 0 deletions pulsar/internal/connection.go
Original file line number Diff line number Diff line change
Expand Up @@ -564,6 +564,9 @@ func (c *connection) internalReceivedCommand(cmd *pb.BaseCommand, headersAndPayl
case pb.BaseCommand_MESSAGE:
c.handleMessage(cmd.GetMessage(), headersAndPayload)

case pb.BaseCommand_ACK_RESPONSE:
c.handleAckResponse(cmd.GetAckResponse())

case pb.BaseCommand_PING:
c.handlePing()
case pb.BaseCommand_PONG:
Expand Down Expand Up @@ -676,6 +679,26 @@ func (c *connection) handleResponseError(serverError *pb.CommandError) {
request.callback(nil, errors.New(errMsg))
}

func (c *connection) handleAckResponse(ackResponse *pb.CommandAckResponse) {
requestID := ackResponse.GetRequestId()
consumerID := ackResponse.GetConsumerId()

request, ok := c.deletePendingRequest(requestID)
if !ok {
c.log.Warnf("AckResponse has complete when receive response! requestId : %d, consumerId : %d",
requestID, consumerID)
return
}

if ackResponse.GetMessage() == "" {
request.callback(nil, nil)
return
}

errMsg := fmt.Sprintf("ack response error: %s: %s", ackResponse.GetError(), ackResponse.GetMessage())
request.callback(nil, errors.New(errMsg))
}

func (c *connection) handleSendReceipt(response *pb.CommandSendReceipt) {
producerID := response.GetProducerId()

Expand Down