Skip to content

Commit 35076ac

Browse files
Support acknowledging a list of message IDs (#1301)
1 parent 875f6ba commit 35076ac

11 files changed

+486
-28
lines changed

pulsar/ack_grouping_tracker.go

Lines changed: 36 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -62,7 +62,7 @@ func newAckGroupingTracker(options *AckGroupingOptions,
6262
maxNumAcks: int(options.MaxSize),
6363
ackCumulative: ackCumulative,
6464
ackList: ackList,
65-
pendingAcks: make(map[[2]uint64]*bitset.BitSet),
65+
pendingAcks: make(map[position]*bitset.BitSet),
6666
lastCumulativeAck: EarliestMessageID(),
6767
}
6868

@@ -110,6 +110,15 @@ func (i *immediateAckGroupingTracker) flushAndClean() {
110110
func (i *immediateAckGroupingTracker) close() {
111111
}
112112

113+
type position struct {
114+
ledgerID uint64
115+
entryID uint64
116+
}
117+
118+
func newPosition(msgID MessageID) position {
119+
return position{ledgerID: uint64(msgID.LedgerID()), entryID: uint64(msgID.EntryID())}
120+
}
121+
113122
type timedAckGroupingTracker struct {
114123
sync.RWMutex
115124

@@ -124,7 +133,7 @@ type timedAckGroupingTracker struct {
124133
// in the batch whose batch size is 3 are not acknowledged.
125134
// After the 1st message (i.e. batch index is 0) is acknowledged, the bits will become "011".
126135
// Value is nil if the entry represents a single message.
127-
pendingAcks map[[2]uint64]*bitset.BitSet
136+
pendingAcks map[position]*bitset.BitSet
128137

129138
lastCumulativeAck MessageID
130139
cumulativeAckRequired int32
@@ -138,35 +147,36 @@ func (t *timedAckGroupingTracker) add(id MessageID) {
138147
}
139148
}
140149

141-
func (t *timedAckGroupingTracker) tryAddIndividual(id MessageID) map[[2]uint64]*bitset.BitSet {
142-
t.Lock()
143-
defer t.Unlock()
144-
key := [2]uint64{uint64(id.LedgerID()), uint64(id.EntryID())}
145-
150+
func addMsgIDToPendingAcks(pendingAcks map[position]*bitset.BitSet, id MessageID) {
151+
key := newPosition(id)
146152
batchIdx := id.BatchIdx()
147153
batchSize := id.BatchSize()
148154

149155
if batchIdx >= 0 && batchSize > 0 {
150-
bs, found := t.pendingAcks[key]
156+
bs, found := pendingAcks[key]
151157
if !found {
152-
if batchSize > 1 {
153-
bs = bitset.New(uint(batchSize))
154-
for i := uint(0); i < uint(batchSize); i++ {
155-
bs.Set(i)
156-
}
158+
bs = bitset.New(uint(batchSize))
159+
for i := uint(0); i < uint(batchSize); i++ {
160+
bs.Set(i)
157161
}
158-
t.pendingAcks[key] = bs
162+
pendingAcks[key] = bs
159163
}
160164
if bs != nil {
161165
bs.Clear(uint(batchIdx))
162166
}
163167
} else {
164-
t.pendingAcks[key] = nil
168+
pendingAcks[key] = nil
165169
}
170+
}
166171

172+
func (t *timedAckGroupingTracker) tryAddIndividual(id MessageID) map[position]*bitset.BitSet {
173+
t.Lock()
174+
defer t.Unlock()
175+
176+
addMsgIDToPendingAcks(t.pendingAcks, id)
167177
if len(t.pendingAcks) >= t.maxNumAcks {
168178
pendingAcks := t.pendingAcks
169-
t.pendingAcks = make(map[[2]uint64]*bitset.BitSet)
179+
t.pendingAcks = make(map[position]*bitset.BitSet)
170180
return pendingAcks
171181
}
172182
return nil
@@ -195,7 +205,7 @@ func (t *timedAckGroupingTracker) isDuplicate(id MessageID) bool {
195205
if messageIDCompare(t.lastCumulativeAck, id) >= 0 {
196206
return true
197207
}
198-
key := [2]uint64{uint64(id.LedgerID()), uint64(id.EntryID())}
208+
key := newPosition(id)
199209
if bs, found := t.pendingAcks[key]; found {
200210
if bs == nil {
201211
return true
@@ -232,11 +242,11 @@ func (t *timedAckGroupingTracker) flushAndClean() {
232242
}
233243
}
234244

235-
func (t *timedAckGroupingTracker) clearPendingAcks() map[[2]uint64]*bitset.BitSet {
245+
func (t *timedAckGroupingTracker) clearPendingAcks() map[position]*bitset.BitSet {
236246
t.Lock()
237247
defer t.Unlock()
238248
pendingAcks := t.pendingAcks
239-
t.pendingAcks = make(map[[2]uint64]*bitset.BitSet)
249+
t.pendingAcks = make(map[position]*bitset.BitSet)
240250
return pendingAcks
241251
}
242252

@@ -250,12 +260,10 @@ func (t *timedAckGroupingTracker) close() {
250260
}
251261
}
252262

253-
func (t *timedAckGroupingTracker) flushIndividual(pendingAcks map[[2]uint64]*bitset.BitSet) {
263+
func toMsgIDDataList(pendingAcks map[position]*bitset.BitSet) []*pb.MessageIdData {
254264
msgIDs := make([]*pb.MessageIdData, 0, len(pendingAcks))
255265
for k, v := range pendingAcks {
256-
ledgerID := k[0]
257-
entryID := k[1]
258-
msgID := &pb.MessageIdData{LedgerId: &ledgerID, EntryId: &entryID}
266+
msgID := &pb.MessageIdData{LedgerId: &k.ledgerID, EntryId: &k.entryID}
259267
if v != nil && !v.None() {
260268
bytes := v.Bytes()
261269
msgID.AckSet = make([]int64, len(bytes))
@@ -265,5 +273,9 @@ func (t *timedAckGroupingTracker) flushIndividual(pendingAcks map[[2]uint64]*bit
265273
}
266274
msgIDs = append(msgIDs, msgID)
267275
}
268-
t.ackList(msgIDs)
276+
return msgIDs
277+
}
278+
279+
func (t *timedAckGroupingTracker) flushIndividual(pendingAcks map[position]*bitset.BitSet) {
280+
t.ackList(toMsgIDDataList(pendingAcks))
269281
}

pulsar/consumer.go

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,8 @@ package pulsar
1919

2020
import (
2121
"context"
22+
"fmt"
23+
"strings"
2224
"time"
2325

2426
"github.com/apache/pulsar-client-go/pulsar/backoff"
@@ -266,6 +268,23 @@ type ConsumerOptions struct {
266268
startMessageID *trackingMessageID
267269
}
268270

271+
// This error is returned when `AckIDList` failed and `AckWithResponse` is true.
272+
// It only contains the valid message IDs that failed to be acknowledged in the `AckIDList` call.
273+
// For those invalid message IDs, users should ignore them and not acknowledge them again.
274+
type AckError map[MessageID]error
275+
276+
func (e AckError) Error() string {
277+
builder := strings.Builder{}
278+
errorMap := make(map[string][]MessageID)
279+
for id, err := range e {
280+
errorMap[err.Error()] = append(errorMap[err.Error()], id)
281+
}
282+
for err, msgIDs := range errorMap {
283+
builder.WriteString(fmt.Sprintf("error: %s, failed message IDs: %v\n", err, msgIDs))
284+
}
285+
return builder.String()
286+
}
287+
269288
// Consumer is an interface that abstracts behavior of Pulsar's consumer
270289
type Consumer interface {
271290
// Subscription get a subscription for the consumer
@@ -305,8 +324,20 @@ type Consumer interface {
305324
Ack(Message) error
306325

307326
// AckID the consumption of a single message, identified by its MessageID
327+
// When `EnableBatchIndexAcknowledgment` is false, if a message ID represents a message in the batch,
328+
// it will not be actually acknowledged by broker until all messages in that batch are acknowledged via
329+
// `AckID` or `AckIDList`.
308330
AckID(MessageID) error
309331

332+
// AckIDList the consumption of a list of messages, identified by their MessageIDs
333+
//
334+
// This method should be used when `AckWithResponse` is true. Otherwise, it will be equivalent with calling
335+
// `AckID` on each message ID in the list.
336+
//
337+
// When `AckWithResponse` is true, the returned error could be an `AckError` which contains the failed message ID
338+
// and the corresponding error.
339+
AckIDList([]MessageID) error
340+
310341
// AckWithTxn the consumption of a single message with a transaction
311342
AckWithTxn(Message, Transaction) error
312343

pulsar/consumer_impl.go

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@ const defaultNackRedeliveryDelay = 1 * time.Minute
4141
type acker interface {
4242
// AckID does not handle errors returned by the Broker side, so no need to wait for doneCh to finish.
4343
AckID(id MessageID) error
44+
AckIDList(msgIDs []MessageID) error
4445
AckIDWithResponse(id MessageID) error
4546
AckIDWithTxn(msgID MessageID, txn Transaction) error
4647
AckIDCumulative(msgID MessageID) error
@@ -559,6 +560,15 @@ func (c *consumer) AckID(msgID MessageID) error {
559560
return c.consumers[msgID.PartitionIdx()].AckID(msgID)
560561
}
561562

563+
func (c *consumer) AckIDList(msgIDs []MessageID) error {
564+
return ackIDListFromMultiTopics(c.log, msgIDs, func(msgID MessageID) (acker, error) {
565+
if err := c.checkMsgIDPartition(msgID); err != nil {
566+
return nil, err
567+
}
568+
return c.consumers[msgID.PartitionIdx()], nil
569+
})
570+
}
571+
562572
// AckCumulative the reception of all the messages in the stream up to (and including)
563573
// the provided message, identified by its MessageID
564574
func (c *consumer) AckCumulative(msg Message) error {

pulsar/consumer_multitopic.go

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -167,6 +167,49 @@ func (c *multiTopicConsumer) AckID(msgID MessageID) error {
167167
return mid.consumer.AckID(msgID)
168168
}
169169

170+
func (c *multiTopicConsumer) AckIDList(msgIDs []MessageID) error {
171+
return ackIDListFromMultiTopics(c.log, msgIDs, func(msgID MessageID) (acker, error) {
172+
if !checkMessageIDType(msgID) {
173+
return nil, fmt.Errorf("invalid message id type %T", msgID)
174+
}
175+
if mid := toTrackingMessageID(msgID); mid != nil && mid.consumer != nil {
176+
return mid.consumer, nil
177+
}
178+
return nil, errors.New("consumer is nil")
179+
})
180+
}
181+
182+
func ackIDListFromMultiTopics(log log.Logger, msgIDs []MessageID, findConsumer func(MessageID) (acker, error)) error {
183+
consumerToMsgIDs := make(map[acker][]MessageID)
184+
for _, msgID := range msgIDs {
185+
if consumer, err := findConsumer(msgID); err == nil {
186+
consumerToMsgIDs[consumer] = append(consumerToMsgIDs[consumer], msgID)
187+
} else {
188+
log.Warnf("Can not find consumer for %v", msgID)
189+
}
190+
}
191+
192+
ackError := AckError{}
193+
for consumer, ids := range consumerToMsgIDs {
194+
if err := consumer.AckIDList(ids); err != nil {
195+
if topicAckError := err.(AckError); topicAckError != nil {
196+
for id, err := range topicAckError {
197+
ackError[id] = err
198+
}
199+
} else {
200+
// It should not reach here
201+
for _, id := range ids {
202+
ackError[id] = err
203+
}
204+
}
205+
}
206+
}
207+
if len(ackError) == 0 {
208+
return nil
209+
}
210+
return ackError
211+
}
212+
170213
// AckWithTxn the consumption of a single message with a transaction
171214
func (c *multiTopicConsumer) AckWithTxn(msg Message, txn Transaction) error {
172215
msgID := msg.ID()

pulsar/consumer_multitopic_test.go

Lines changed: 99 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import (
2121
"fmt"
2222
"strings"
2323
"testing"
24+
"time"
2425

2526
"github.com/apache/pulsar-client-go/pulsaradmin"
2627
"github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config"
@@ -218,3 +219,101 @@ func TestMultiTopicGetLastMessageIDs(t *testing.T) {
218219
}
219220

220221
}
222+
223+
func TestMultiTopicAckIDList(t *testing.T) {
224+
for _, params := range []bool{true, false} {
225+
t.Run(fmt.Sprintf("TestMultiTopicConsumerAckIDList%v", params), func(t *testing.T) {
226+
runMultiTopicAckIDList(t, params)
227+
})
228+
}
229+
}
230+
231+
func runMultiTopicAckIDList(t *testing.T, regex bool) {
232+
topicPrefix := fmt.Sprintf("multiTopicAckIDList%v", time.Now().UnixNano())
233+
topic1 := "persistent://public/default/" + topicPrefix + "1"
234+
topic2 := "persistent://public/default/" + topicPrefix + "2"
235+
236+
client, err := NewClient(ClientOptions{URL: "pulsar://localhost:6650"})
237+
assert.Nil(t, err)
238+
defer client.Close()
239+
240+
if regex {
241+
admin, err := pulsaradmin.NewClient(&config.Config{})
242+
assert.Nil(t, err)
243+
for _, topic := range []string{topic1, topic2} {
244+
topicName, err := utils.GetTopicName(topic)
245+
assert.Nil(t, err)
246+
admin.Topics().Create(*topicName, 0)
247+
}
248+
}
249+
250+
createConsumer := func() Consumer {
251+
options := ConsumerOptions{
252+
SubscriptionName: "sub",
253+
Type: Shared,
254+
AckWithResponse: true,
255+
}
256+
if regex {
257+
options.TopicsPattern = topicPrefix + ".*"
258+
} else {
259+
options.Topics = []string{topic1, topic2}
260+
}
261+
consumer, err := client.Subscribe(options)
262+
assert.Nil(t, err)
263+
return consumer
264+
}
265+
consumer := createConsumer()
266+
267+
sendMessages(t, client, topic1, 0, 3, false)
268+
sendMessages(t, client, topic2, 0, 2, false)
269+
270+
receiveMessageMap := func(consumer Consumer, numMessages int) map[string][]Message {
271+
msgs := receiveMessages(t, consumer, numMessages)
272+
topicToMsgs := make(map[string][]Message)
273+
for _, msg := range msgs {
274+
topicToMsgs[msg.Topic()] = append(topicToMsgs[msg.Topic()], msg)
275+
}
276+
return topicToMsgs
277+
}
278+
279+
topicToMsgs := receiveMessageMap(consumer, 5)
280+
assert.Equal(t, 3, len(topicToMsgs[topic1]))
281+
for i := 0; i < 3; i++ {
282+
assert.Equal(t, fmt.Sprintf("msg-%d", i), string(topicToMsgs[topic1][i].Payload()))
283+
}
284+
assert.Equal(t, 2, len(topicToMsgs[topic2]))
285+
for i := 0; i < 2; i++ {
286+
assert.Equal(t, fmt.Sprintf("msg-%d", i), string(topicToMsgs[topic2][i].Payload()))
287+
}
288+
289+
assert.Nil(t, consumer.AckIDList([]MessageID{
290+
topicToMsgs[topic1][0].ID(),
291+
topicToMsgs[topic1][2].ID(),
292+
topicToMsgs[topic2][1].ID(),
293+
}))
294+
295+
consumer.Close()
296+
consumer = createConsumer()
297+
topicToMsgs = receiveMessageMap(consumer, 2)
298+
assert.Equal(t, 1, len(topicToMsgs[topic1]))
299+
assert.Equal(t, "msg-1", string(topicToMsgs[topic1][0].Payload()))
300+
assert.Equal(t, 1, len(topicToMsgs[topic2]))
301+
assert.Equal(t, "msg-0", string(topicToMsgs[topic2][0].Payload()))
302+
consumer.Close()
303+
304+
msgID0 := topicToMsgs[topic1][0].ID()
305+
err = consumer.AckIDList([]MessageID{msgID0})
306+
assert.NotNil(t, err)
307+
t.Logf("AckIDList error: %v", err)
308+
309+
msgID1 := topicToMsgs[topic2][0].ID()
310+
if ackError, ok := consumer.AckIDList([]MessageID{msgID0, msgID1}).(AckError); ok {
311+
assert.Equal(t, 2, len(ackError))
312+
assert.Contains(t, ackError, msgID0)
313+
assert.Equal(t, "consumer state is closed", ackError[msgID0].Error())
314+
assert.Contains(t, ackError, msgID1)
315+
assert.Equal(t, "consumer state is closed", ackError[msgID1].Error())
316+
} else {
317+
assert.Fail(t, "AckIDList should return AckError")
318+
}
319+
}

0 commit comments

Comments
 (0)