Skip to content

Commit ffdc3af

Browse files
Improve the AckIDList performance when there are many topics subscribed (#1305)
### Motivation Currently when a consumer subscribes multiple topic-partitions and `AckWithResponse` is true, the `AckIDList` method will iterate over all internal consumers **sequentially**. It harms the performance especially there are many internal consumers. For example, if the connection of an internal consumer was stuck by some reason, message IDs from other consumer would be blocked for the operation timeout. ### Modifications In `ackIDListFromMultiTopics`, call `consumer.AckIDList` in goroutines and use a channel to receive all errors from these calls. Add `TestMultiTopicAckIDListTimeout`, which sets a dummy connection instance whose `SendRequest` never completes the callback, to verify the `AckIDList` call will not take much more time than the operation timeout to complete. Without this improvement, it will take more than 5 times of the operation timeout to fail.
1 parent a144d88 commit ffdc3af

File tree

3 files changed

+114
-15
lines changed

3 files changed

+114
-15
lines changed

pulsar/consumer_multitopic.go

Lines changed: 10 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -189,18 +189,17 @@ func ackIDListFromMultiTopics(log log.Logger, msgIDs []MessageID, findConsumer f
189189
}
190190
}
191191

192-
ackError := AckError{}
192+
subErrCh := make(chan error, len(consumerToMsgIDs))
193193
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-
}
194+
go func() {
195+
subErrCh <- consumer.AckIDList(ids)
196+
}()
197+
}
198+
ackError := AckError{}
199+
for i := 0; i < len(consumerToMsgIDs); i++ {
200+
if topicAckError, ok := (<-subErrCh).(AckError); ok {
201+
for id, err := range topicAckError {
202+
ackError[id] = err
204203
}
205204
}
206205
}

pulsar/consumer_multitopic_test.go

Lines changed: 101 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,15 +18,17 @@
1818
package pulsar
1919

2020
import (
21+
"errors"
2122
"fmt"
2223
"strings"
2324
"testing"
2425
"time"
2526

27+
"github.com/apache/pulsar-client-go/pulsar/internal"
28+
pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto"
2629
"github.com/apache/pulsar-client-go/pulsaradmin"
2730
"github.com/apache/pulsar-client-go/pulsaradmin/pkg/admin/config"
2831
"github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils"
29-
3032
"github.com/stretchr/testify/assert"
3133
)
3234

@@ -317,3 +319,101 @@ func runMultiTopicAckIDList(t *testing.T, regex bool) {
317319
assert.Fail(t, "AckIDList should return AckError")
318320
}
319321
}
322+
323+
type dummyConnection struct {
324+
}
325+
326+
func (dummyConnection) SendRequest(_ uint64, _ *pb.BaseCommand, _ func(*pb.BaseCommand, error)) {
327+
}
328+
329+
func (dummyConnection) SendRequestNoWait(_ *pb.BaseCommand) error {
330+
return nil
331+
}
332+
333+
func (dummyConnection) WriteData(_ internal.Buffer) {
334+
}
335+
336+
func (dummyConnection) RegisterListener(_ uint64, _ internal.ConnectionListener) error {
337+
return nil
338+
}
339+
340+
func (dummyConnection) UnregisterListener(_ uint64) {
341+
}
342+
343+
func (dummyConnection) AddConsumeHandler(_ uint64, _ internal.ConsumerHandler) error {
344+
return nil
345+
}
346+
347+
func (dummyConnection) DeleteConsumeHandler(_ uint64) {
348+
}
349+
350+
func (dummyConnection) ID() string {
351+
return "cnx"
352+
}
353+
354+
func (dummyConnection) GetMaxMessageSize() int32 {
355+
return 0
356+
}
357+
358+
func (dummyConnection) Close() {
359+
}
360+
361+
func (dummyConnection) WaitForClose() <-chan struct{} {
362+
return nil
363+
}
364+
365+
func (dummyConnection) IsProxied() bool {
366+
return false
367+
}
368+
369+
func TestMultiTopicAckIDListTimeout(t *testing.T) {
370+
topic := fmt.Sprintf("multiTopicAckIDListTimeout%v", time.Now().UnixNano())
371+
assert.NoError(t, createPartitionedTopic(topic, 5))
372+
373+
cli, err := NewClient(ClientOptions{
374+
URL: "pulsar://localhost:6650",
375+
OperationTimeout: 3 * time.Second,
376+
})
377+
assert.Nil(t, err)
378+
defer cli.Close()
379+
380+
consumer, err := cli.Subscribe(ConsumerOptions{
381+
Topic: topic,
382+
SubscriptionName: "sub",
383+
AckWithResponse: true,
384+
})
385+
assert.Nil(t, err)
386+
defer consumer.Close()
387+
388+
const numMessages = 5
389+
sendMessages(t, cli, topic, 0, numMessages, false)
390+
msgs := receiveMessages(t, consumer, numMessages)
391+
msgIDs := make([]MessageID, len(msgs))
392+
393+
var conn internal.Connection
394+
for i := 0; i < len(msgs); i++ {
395+
msgIDs[i] = msgs[i].ID()
396+
pc, ok := msgIDs[i].(*trackingMessageID).consumer.(*partitionConsumer)
397+
assert.True(t, ok)
398+
conn = pc._getConn()
399+
pc._setConn(dummyConnection{})
400+
}
401+
402+
start := time.Now()
403+
err = consumer.AckIDList(msgIDs)
404+
elapsed := time.Since(start)
405+
t.Logf("AckIDList takes %v ms", elapsed)
406+
assert.True(t, elapsed < 5*time.Second && elapsed >= 3*time.Second)
407+
var ackError AckError
408+
if errors.As(err, &ackError) {
409+
for _, err := range ackError {
410+
assert.Equal(t, "request timed out", err.Error())
411+
}
412+
}
413+
414+
for i := 0; i < len(msgs); i++ {
415+
pc, ok := msgIDs[i].(*trackingMessageID).consumer.(*partitionConsumer)
416+
assert.True(t, ok)
417+
pc._setConn(conn)
418+
}
419+
}

pulsar/consumer_partition.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -141,7 +141,7 @@ type partitionConsumer struct {
141141
state uAtomic.Int32
142142
options *partitionConsumerOpts
143143

144-
conn uAtomic.Value
144+
conn atomic.Pointer[internal.Connection]
145145

146146
topic string
147147
name string
@@ -2205,7 +2205,7 @@ func (pc *partitionConsumer) hasMoreMessages() bool {
22052205
// _setConn sets the internal connection field of this partition consumer atomically.
22062206
// Note: should only be called by this partition consumer when a new connection is available.
22072207
func (pc *partitionConsumer) _setConn(conn internal.Connection) {
2208-
pc.conn.Store(conn)
2208+
pc.conn.Store(&conn)
22092209
}
22102210

22112211
// _getConn returns internal connection field of this partition consumer atomically.
@@ -2214,7 +2214,7 @@ func (pc *partitionConsumer) _getConn() internal.Connection {
22142214
// Invariant: The conn must be non-nill for the lifetime of the partitionConsumer.
22152215
// For this reason we leave this cast unchecked and panic() if the
22162216
// invariant is broken
2217-
return pc.conn.Load().(internal.Connection)
2217+
return *pc.conn.Load()
22182218
}
22192219

22202220
func convertToMessageIDData(msgID *trackingMessageID) *pb.MessageIdData {

0 commit comments

Comments
 (0)