-
Notifications
You must be signed in to change notification settings - Fork 336
/
consumer_partition.go
2253 lines (1955 loc) · 64.3 KB
/
consumer_partition.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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 (
"container/list"
"encoding/hex"
"errors"
"fmt"
"math"
"strings"
"sync"
"time"
"google.golang.org/protobuf/proto"
"github.com/apache/pulsar-client-go/pulsar/crypto"
"github.com/apache/pulsar-client-go/pulsar/internal"
"github.com/apache/pulsar-client-go/pulsar/internal/compression"
cryptointernal "github.com/apache/pulsar-client-go/pulsar/internal/crypto"
pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto"
"github.com/apache/pulsar-client-go/pulsar/log"
"github.com/bits-and-blooms/bitset"
uAtomic "go.uber.org/atomic"
)
type consumerState int
const (
// consumer states
consumerInit = iota
consumerReady
consumerClosing
consumerClosed
)
func (s consumerState) String() string {
switch s {
case consumerInit:
return "Initializing"
case consumerReady:
return "Ready"
case consumerClosing:
return "Closing"
case consumerClosed:
return "Closed"
default:
return "Unknown"
}
}
type SubscriptionMode int
const (
// Make the subscription to be backed by a durable cursor that will retain messages and persist the current
// position
Durable SubscriptionMode = iota
// Lightweight subscription mode that doesn't have a durable cursor associated
NonDurable
)
const (
initialReceiverQueueSize = 1
receiverQueueExpansionMemThreshold = 0.75
)
const (
noMessageEntry = -1
)
type partitionConsumerOpts struct {
topic string
consumerName string
subscription string
subscriptionType SubscriptionType
subscriptionInitPos SubscriptionInitialPosition
partitionIdx int
receiverQueueSize int
autoReceiverQueueSize bool
nackRedeliveryDelay time.Duration
nackBackoffPolicy NackBackoffPolicy
metadata map[string]string
subProperties map[string]string
replicateSubscriptionState bool
startMessageID *trackingMessageID
startMessageIDInclusive bool
subscriptionMode SubscriptionMode
readCompacted bool
disableForceTopicCreation bool
interceptors ConsumerInterceptors
maxReconnectToBroker *uint
backoffPolicy internal.BackoffPolicy
keySharedPolicy *KeySharedPolicy
schema Schema
decryption *MessageDecryptionInfo
ackWithResponse bool
maxPendingChunkedMessage int
expireTimeOfIncompleteChunk time.Duration
autoAckIncompleteChunk bool
// in failover mode, this callback will be called when consumer change
consumerEventListener ConsumerEventListener
enableBatchIndexAck bool
ackGroupingOptions *AckGroupingOptions
}
type ConsumerEventListener interface {
BecameActive(consumer Consumer, topicName string, partition int32)
BecameInactive(consumer Consumer, topicName string, partition int32)
}
type partitionConsumer struct {
client *client
// this is needed for sending ConsumerMessage on the messageCh
parentConsumer Consumer
state uAtomic.Int32
options *partitionConsumerOpts
conn uAtomic.Value
topic string
name string
consumerID uint64
partitionIdx int32
// shared channel
messageCh chan ConsumerMessage
// the number of message slots available
availablePermits *availablePermits
// the size of the queue channel for buffering messages
maxQueueSize int32
queueCh chan []*message
startMessageID atomicMessageID
lastDequeuedMsg *trackingMessageID
currentQueueSize uAtomic.Int32
scaleReceiverQueueHint uAtomic.Bool
incomingMessages uAtomic.Int32
eventsCh chan interface{}
connectedCh chan struct{}
connectClosedCh chan connectionClosed
closeCh chan struct{}
clearQueueCh chan func(id *trackingMessageID)
nackTracker *negativeAcksTracker
dlq *dlqRouter
log log.Logger
compressionProviders sync.Map //map[pb.CompressionType]compression.Provider
metrics *internal.LeveledMetrics
decryptor cryptointernal.Decryptor
schemaInfoCache *schemaInfoCache
chunkedMsgCtxMap *chunkedMsgCtxMap
unAckChunksTracker *unAckChunksTracker
ackGroupingTracker ackGroupingTracker
}
func (pc *partitionConsumer) ActiveConsumerChanged(isActive bool) {
listener := pc.options.consumerEventListener
if listener == nil {
// didn't set a listener
return
}
if isActive {
listener.BecameActive(pc.parentConsumer, pc.topic, pc.partitionIdx)
} else {
listener.BecameInactive(pc.parentConsumer, pc.topic, pc.partitionIdx)
}
}
type availablePermits struct {
permits uAtomic.Int32
pc *partitionConsumer
}
func (p *availablePermits) inc() {
// atomic add availablePermits
p.add(1)
}
func (p *availablePermits) add(delta int32) {
p.permits.Add(delta)
p.flowIfNeed()
}
func (p *availablePermits) reset() {
p.permits.Store(0)
}
func (p *availablePermits) get() int32 {
return p.permits.Load()
}
func (p *availablePermits) flowIfNeed() {
// TODO implement a better flow controller
// send more permits if needed
var flowThreshold int32
if p.pc.options.autoReceiverQueueSize {
flowThreshold = int32(math.Max(float64(p.pc.currentQueueSize.Load()/2), 1))
} else {
flowThreshold = int32(math.Max(float64(p.pc.maxQueueSize/2), 1))
}
current := p.get()
if current >= flowThreshold {
availablePermits := current
requestedPermits := current
// check if permits changed
if !p.permits.CAS(current, 0) {
return
}
p.pc.log.Debugf("requesting more permits=%d available=%d", requestedPermits, availablePermits)
if err := p.pc.internalFlow(uint32(requestedPermits)); err != nil {
p.pc.log.WithError(err).Error("unable to send permits")
}
}
}
// atomicMessageID is a wrapper for trackingMessageID to make get and set atomic
type atomicMessageID struct {
msgID *trackingMessageID
sync.RWMutex
}
func (a *atomicMessageID) get() *trackingMessageID {
a.RLock()
defer a.RUnlock()
return a.msgID
}
func (a *atomicMessageID) set(msgID *trackingMessageID) {
a.Lock()
defer a.Unlock()
a.msgID = msgID
}
type schemaInfoCache struct {
lock sync.RWMutex
cache map[string]Schema
client *client
topic string
}
func newSchemaInfoCache(client *client, topic string) *schemaInfoCache {
return &schemaInfoCache{
cache: make(map[string]Schema),
client: client,
topic: topic,
}
}
func (s *schemaInfoCache) Get(schemaVersion []byte) (schema Schema, err error) {
key := hex.EncodeToString(schemaVersion)
s.lock.RLock()
schema, ok := s.cache[key]
s.lock.RUnlock()
if ok {
return schema, nil
}
pbSchema, err := s.client.lookupService.GetSchema(s.topic, schemaVersion)
if err != nil {
return nil, err
}
if pbSchema == nil {
err = fmt.Errorf("schema not found for topic: [ %v ], schema version : [ %v ]", s.topic, schemaVersion)
return nil, err
}
var properties = internal.ConvertToStringMap(pbSchema.Properties)
schema, err = NewSchema(SchemaType(*pbSchema.Type), pbSchema.SchemaData, properties)
if err != nil {
return nil, err
}
s.add(key, schema)
return schema, nil
}
func (s *schemaInfoCache) add(schemaVersionHash string, schema Schema) {
s.lock.Lock()
defer s.lock.Unlock()
s.cache[schemaVersionHash] = schema
}
func newPartitionConsumer(parent Consumer, client *client, options *partitionConsumerOpts,
messageCh chan ConsumerMessage, dlq *dlqRouter,
metrics *internal.LeveledMetrics) (*partitionConsumer, error) {
pc := &partitionConsumer{
parentConsumer: parent,
client: client,
options: options,
topic: options.topic,
name: options.consumerName,
consumerID: client.rpcClient.NewConsumerID(),
partitionIdx: int32(options.partitionIdx),
eventsCh: make(chan interface{}, 10),
maxQueueSize: int32(options.receiverQueueSize),
queueCh: make(chan []*message, options.receiverQueueSize),
startMessageID: atomicMessageID{msgID: options.startMessageID},
connectedCh: make(chan struct{}),
messageCh: messageCh,
connectClosedCh: make(chan connectionClosed, 10),
closeCh: make(chan struct{}),
clearQueueCh: make(chan func(id *trackingMessageID)),
compressionProviders: sync.Map{},
dlq: dlq,
metrics: metrics,
schemaInfoCache: newSchemaInfoCache(client, options.topic),
}
if pc.options.autoReceiverQueueSize {
pc.currentQueueSize.Store(initialReceiverQueueSize)
pc.client.memLimit.RegisterTrigger(pc.shrinkReceiverQueueSize)
} else {
pc.currentQueueSize.Store(int32(pc.options.receiverQueueSize))
}
pc.availablePermits = &availablePermits{pc: pc}
pc.chunkedMsgCtxMap = newChunkedMsgCtxMap(options.maxPendingChunkedMessage, pc)
pc.unAckChunksTracker = newUnAckChunksTracker(pc)
pc.ackGroupingTracker = newAckGroupingTracker(options.ackGroupingOptions,
func(id MessageID) { pc.sendIndividualAck(id) },
func(id MessageID) { pc.sendCumulativeAck(id) },
func(ids []*pb.MessageIdData) { pc.eventsCh <- ids })
pc.setConsumerState(consumerInit)
pc.log = client.log.SubLogger(log.Fields{
"name": pc.name,
"topic": options.topic,
"subscription": options.subscription,
"consumerID": pc.consumerID,
})
var decryptor cryptointernal.Decryptor
if pc.options.decryption == nil {
decryptor = cryptointernal.NewNoopDecryptor() // default to noopDecryptor
} else {
decryptor = cryptointernal.NewConsumerDecryptor(
options.decryption.KeyReader,
options.decryption.MessageCrypto,
pc.log,
)
}
pc.decryptor = decryptor
pc.nackTracker = newNegativeAcksTracker(pc, options.nackRedeliveryDelay, options.nackBackoffPolicy, pc.log)
err := pc.grabConn()
if err != nil {
pc.log.WithError(err).Error("Failed to create consumer")
pc.nackTracker.Close()
pc.ackGroupingTracker.close()
pc.chunkedMsgCtxMap.Close()
return nil, err
}
pc.log.Info("Created consumer")
pc.setConsumerState(consumerReady)
startingMessageID := pc.startMessageID.get()
if pc.options.startMessageIDInclusive && startingMessageID != nil && startingMessageID.equal(latestMessageID) {
msgID, err := pc.requestGetLastMessageID()
if err != nil {
pc.Close()
return nil, err
}
if msgID.entryID != noMessageEntry {
pc.startMessageID.set(msgID)
// use the WithoutClear version because the dispatcher is not started yet
err = pc.requestSeekWithoutClear(msgID.messageID)
if err != nil {
pc.Close()
return nil, err
}
}
}
go pc.dispatcher()
go pc.runEventsLoop()
return pc, nil
}
func (pc *partitionConsumer) Unsubscribe() error {
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to unsubscribe closing or closed consumer")
return nil
}
req := &unsubscribeRequest{doneCh: make(chan struct{})}
pc.eventsCh <- req
// wait for the request to complete
<-req.doneCh
return req.err
}
// ackIDCommon handles common logic for acknowledging messages with or without transactions.
// withTxn should be set to true when dealing with transactions.
func (pc *partitionConsumer) ackIDCommon(msgID MessageID, withResponse bool, txn Transaction) 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")
}
if cmid, ok := msgID.(*chunkMessageID); ok {
if txn == nil {
return pc.unAckChunksTracker.ack(cmid)
}
return pc.unAckChunksTracker.ackWithTxn(cmid, txn)
}
trackingID := toTrackingMessageID(msgID)
if trackingID != nil && trackingID.ack() {
// All messages in the same batch have been acknowledged, we only need to acknowledge the
// MessageID that represents the entry that stores the whole batch
trackingID = &trackingMessageID{
messageID: &messageID{
ledgerID: trackingID.ledgerID,
entryID: trackingID.entryID,
},
}
pc.metrics.AcksCounter.Inc()
pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9)
} else if !pc.options.enableBatchIndexAck {
return nil
}
var err error
if withResponse {
if txn != nil {
ackReq := pc.sendIndividualAckWithTxn(trackingID, txn.(*transaction))
<-ackReq.doneCh
err = ackReq.err
} else {
ackReq := pc.sendIndividualAck(trackingID)
<-ackReq.doneCh
err = ackReq.err
}
} else {
pc.ackGroupingTracker.add(trackingID)
}
pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID)
return err
}
// AckIDWithTxn acknowledges the consumption of a message with transaction.
func (pc *partitionConsumer) AckIDWithTxn(msgID MessageID, txn Transaction) error {
return pc.ackIDCommon(msgID, true, txn)
}
// ackID acknowledges the consumption of a message and optionally waits for response from the broker.
func (pc *partitionConsumer) ackID(msgID MessageID, withResponse bool) error {
return pc.ackIDCommon(msgID, withResponse, nil)
}
func (pc *partitionConsumer) internalAckWithTxn(req *ackWithTxnRequest) {
defer close(req.doneCh)
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer")
req.err = newError(ConsumerClosed, "Failed to ack by closing or closed consumer")
return
}
if req.Transaction.state != TxnOpen {
pc.log.WithField("state", req.Transaction.state).Error("Failed to ack by a non-open transaction.")
req.err = newError(InvalidStatus, "Failed to ack by a non-open transaction.")
return
}
msgID := req.msgID
messageIDs := make([]*pb.MessageIdData, 1)
messageIDs[0] = &pb.MessageIdData{
LedgerId: proto.Uint64(uint64(msgID.ledgerID)),
EntryId: proto.Uint64(uint64(msgID.entryID)),
}
if pc.options.enableBatchIndexAck && msgID.tracker != nil {
ackSet := msgID.tracker.toAckSet()
if ackSet != nil {
messageIDs[0].AckSet = ackSet
}
}
reqID := pc.client.rpcClient.NewRequestID()
txnID := req.Transaction.GetTxnID()
cmdAck := &pb.CommandAck{
ConsumerId: proto.Uint64(pc.consumerID),
MessageId: messageIDs,
AckType: pb.CommandAck_Individual.Enum(),
TxnidMostBits: proto.Uint64(txnID.MostSigBits),
TxnidLeastBits: proto.Uint64(txnID.LeastSigBits),
}
if err := req.Transaction.registerAckTopic(pc.options.topic, pc.options.subscription); err != nil {
req.err = err
return
}
if err := req.Transaction.registerSendOrAckOp(); err != nil {
req.err = err
return
}
cmdAck.RequestId = proto.Uint64(reqID)
_, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), reqID, pb.BaseCommand_ACK, cmdAck)
if err != nil {
pc.log.WithError(err).Error("Ack with response error")
}
req.Transaction.endSendOrAckOp(err)
req.err = err
}
func (pc *partitionConsumer) internalUnsubscribe(unsub *unsubscribeRequest) {
defer close(unsub.doneCh)
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to unsubscribe closing or closed consumer")
return
}
pc.setConsumerState(consumerClosing)
requestID := pc.client.rpcClient.NewRequestID()
cmdUnsubscribe := &pb.CommandUnsubscribe{
RequestId: proto.Uint64(requestID),
ConsumerId: proto.Uint64(pc.consumerID),
}
_, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), requestID, pb.BaseCommand_UNSUBSCRIBE, cmdUnsubscribe)
if err != nil {
pc.log.WithError(err).Error("Failed to unsubscribe consumer")
unsub.err = err
// Set the state to ready for closing the consumer
pc.setConsumerState(consumerReady)
// Should'nt remove the consumer handler
return
}
pc._getConn().DeleteConsumeHandler(pc.consumerID)
if pc.nackTracker != nil {
pc.nackTracker.Close()
}
pc.log.Infof("The consumer[%d] successfully unsubscribed", pc.consumerID)
pc.setConsumerState(consumerClosed)
}
func (pc *partitionConsumer) getLastMessageID() (*trackingMessageID, error) {
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to redeliver closing or closed consumer")
return nil, errors.New("failed to redeliver closing or closed consumer")
}
req := &getLastMsgIDRequest{doneCh: make(chan struct{})}
pc.eventsCh <- req
// wait for the request to complete
<-req.doneCh
return req.msgID, req.err
}
func (pc *partitionConsumer) internalGetLastMessageID(req *getLastMsgIDRequest) {
defer close(req.doneCh)
req.msgID, req.err = pc.requestGetLastMessageID()
}
func (pc *partitionConsumer) requestGetLastMessageID() (*trackingMessageID, error) {
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to getLastMessageID closing or closed consumer")
return nil, errors.New("failed to getLastMessageID closing or closed consumer")
}
requestID := pc.client.rpcClient.NewRequestID()
cmdGetLastMessageID := &pb.CommandGetLastMessageId{
RequestId: proto.Uint64(requestID),
ConsumerId: proto.Uint64(pc.consumerID),
}
res, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), requestID,
pb.BaseCommand_GET_LAST_MESSAGE_ID, cmdGetLastMessageID)
if err != nil {
pc.log.WithError(err).Error("Failed to get last message id")
return nil, err
}
id := res.Response.GetLastMessageIdResponse.GetLastMessageId()
return convertToMessageID(id), nil
}
func (pc *partitionConsumer) sendIndividualAck(msgID MessageID) *ackRequest {
ackReq := &ackRequest{
doneCh: make(chan struct{}),
ackType: individualAck,
msgID: *msgID.(*trackingMessageID),
}
pc.eventsCh <- ackReq
return ackReq
}
func (pc *partitionConsumer) sendIndividualAckWithTxn(msgID MessageID, txn *transaction) *ackWithTxnRequest {
ackReq := &ackWithTxnRequest{
Transaction: txn,
doneCh: make(chan struct{}),
ackType: individualAck,
msgID: *msgID.(*trackingMessageID),
}
pc.eventsCh <- ackReq
return ackReq
}
func (pc *partitionConsumer) AckIDWithResponse(msgID MessageID) error {
if !checkMessageIDType(msgID) {
pc.log.Errorf("invalid message id type %T", msgID)
return fmt.Errorf("invalid message id type %T", msgID)
}
return pc.ackID(msgID, true)
}
func (pc *partitionConsumer) AckID(msgID MessageID) error {
if !checkMessageIDType(msgID) {
pc.log.Errorf("invalid message id type %T", msgID)
return fmt.Errorf("invalid message id type %T", msgID)
}
return pc.ackID(msgID, false)
}
func (pc *partitionConsumer) AckIDCumulative(msgID MessageID) error {
if !checkMessageIDType(msgID) {
pc.log.Errorf("invalid message id type %T", msgID)
return fmt.Errorf("invalid message id type %T", msgID)
}
return pc.internalAckIDCumulative(msgID, false)
}
func (pc *partitionConsumer) AckIDWithResponseCumulative(msgID MessageID) error {
if !checkMessageIDType(msgID) {
pc.log.Errorf("invalid message id type %T", msgID)
return fmt.Errorf("invalid message id type %T", msgID)
}
return pc.internalAckIDCumulative(msgID, true)
}
func (pc *partitionConsumer) internalAckIDCumulative(msgID MessageID, withResponse bool) 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 := toTrackingMessageID(msgID)
if trackingID == nil {
return errors.New("failed to convert trackingMessageID")
}
var msgIDToAck *trackingMessageID
if trackingID.ackCumulative() || pc.options.enableBatchIndexAck {
msgIDToAck = trackingID
} else if !trackingID.tracker.hasPrevBatchAcked() {
// get previous batch message id
msgIDToAck = trackingID.prev()
trackingID.tracker.setPrevBatchAcked()
} else {
// waiting for all the msgs are acked in this batch
return nil
}
pc.metrics.AcksCounter.Inc()
pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9)
var ackReq *ackRequest
if withResponse {
ackReq := pc.sendCumulativeAck(msgIDToAck)
<-ackReq.doneCh
} else {
pc.ackGroupingTracker.addCumulative(msgIDToAck)
}
pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID)
if cmid, ok := msgID.(*chunkMessageID); ok {
pc.unAckChunksTracker.remove(cmid)
}
if ackReq == nil {
return nil
}
return ackReq.err
}
func (pc *partitionConsumer) sendCumulativeAck(msgID MessageID) *ackRequest {
ackReq := &ackRequest{
doneCh: make(chan struct{}),
ackType: cumulativeAck,
msgID: *msgID.(*trackingMessageID),
}
pc.eventsCh <- ackReq
return ackReq
}
func (pc *partitionConsumer) NackID(msgID MessageID) {
if !checkMessageIDType(msgID) {
pc.log.Warnf("invalid message id type %T", msgID)
return
}
if cmid, ok := msgID.(*chunkMessageID); ok {
pc.unAckChunksTracker.nack(cmid)
return
}
trackingID := toTrackingMessageID(msgID)
pc.nackTracker.Add(trackingID.messageID)
pc.metrics.NacksCounter.Inc()
}
func (pc *partitionConsumer) NackMsg(msg Message) {
pc.nackTracker.AddMessage(msg)
pc.metrics.NacksCounter.Inc()
}
func (pc *partitionConsumer) Redeliver(msgIds []messageID) {
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to redeliver closing or closed consumer")
return
}
pc.eventsCh <- &redeliveryRequest{msgIds}
iMsgIds := make([]MessageID, len(msgIds))
for i := range iMsgIds {
iMsgIds[i] = &msgIds[i]
}
pc.options.interceptors.OnNegativeAcksSend(pc.parentConsumer, iMsgIds)
}
func (pc *partitionConsumer) internalRedeliver(req *redeliveryRequest) {
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to redeliver closing or closed consumer")
return
}
msgIds := req.msgIds
pc.log.Debug("Request redelivery after negative ack for messages", msgIds)
msgIDDataList := make([]*pb.MessageIdData, len(msgIds))
for i := 0; i < len(msgIds); i++ {
msgIDDataList[i] = &pb.MessageIdData{
LedgerId: proto.Uint64(uint64(msgIds[i].ledgerID)),
EntryId: proto.Uint64(uint64(msgIds[i].entryID)),
}
}
err := pc.client.rpcClient.RequestOnCnxNoWait(pc._getConn(),
pb.BaseCommand_REDELIVER_UNACKNOWLEDGED_MESSAGES, &pb.CommandRedeliverUnacknowledgedMessages{
ConsumerId: proto.Uint64(pc.consumerID),
MessageIds: msgIDDataList,
})
if err != nil {
pc.log.Error("Connection was closed when request redeliver cmd")
}
}
func (pc *partitionConsumer) getConsumerState() consumerState {
return consumerState(pc.state.Load())
}
func (pc *partitionConsumer) setConsumerState(state consumerState) {
pc.state.Store(int32(state))
}
func (pc *partitionConsumer) Close() {
if pc.getConsumerState() != consumerReady {
return
}
// flush all pending ACK requests and terminate the timer goroutine
pc.ackGroupingTracker.close()
// close chunkedMsgCtxMap
pc.chunkedMsgCtxMap.Close()
req := &closeRequest{doneCh: make(chan struct{})}
pc.eventsCh <- req
// wait for request to finish
<-req.doneCh
}
func (pc *partitionConsumer) Seek(msgID MessageID) error {
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to seek by closing or closed consumer")
return errors.New("failed to seek by closing or closed consumer")
}
if !checkMessageIDType(msgID) {
pc.log.Errorf("invalid message id type %T", msgID)
return fmt.Errorf("invalid message id type %T", msgID)
}
req := &seekRequest{
doneCh: make(chan struct{}),
}
if cmid, ok := msgID.(*chunkMessageID); ok {
req.msgID = cmid.firstChunkID
} else {
tmid := toTrackingMessageID(msgID)
req.msgID = tmid.messageID
}
pc.ackGroupingTracker.flushAndClean()
pc.eventsCh <- req
// wait for the request to complete
<-req.doneCh
return req.err
}
func (pc *partitionConsumer) internalSeek(seek *seekRequest) {
defer close(seek.doneCh)
seek.err = pc.requestSeek(seek.msgID)
}
func (pc *partitionConsumer) requestSeek(msgID *messageID) error {
if err := pc.requestSeekWithoutClear(msgID); err != nil {
return err
}
pc.clearReceiverQueue()
return nil
}
func (pc *partitionConsumer) requestSeekWithoutClear(msgID *messageID) error {
state := pc.getConsumerState()
if state == consumerClosing || state == consumerClosed {
pc.log.WithField("state", state).Error("failed seek by consumer is closing or has closed")
return nil
}
id := &pb.MessageIdData{}
err := proto.Unmarshal(msgID.Serialize(), id)
if err != nil {
pc.log.WithError(err).Errorf("deserialize message id error: %s", err.Error())
return err
}
requestID := pc.client.rpcClient.NewRequestID()
cmdSeek := &pb.CommandSeek{
ConsumerId: proto.Uint64(pc.consumerID),
RequestId: proto.Uint64(requestID),
MessageId: id,
}
_, err = pc.client.rpcClient.RequestOnCnx(pc._getConn(), requestID, pb.BaseCommand_SEEK, cmdSeek)
if err != nil {
pc.log.WithError(err).Error("Failed to reset to message id")
return err
}
return nil
}
func (pc *partitionConsumer) SeekByTime(time time.Time) error {
if state := pc.getConsumerState(); state == consumerClosing || state == consumerClosed {
pc.log.WithField("state", pc.state).Error("Failed seekByTime by consumer is closing or has closed")
return errors.New("failed seekByTime by consumer is closing or has closed")
}
req := &seekByTimeRequest{
doneCh: make(chan struct{}),
publishTime: time,
}
pc.ackGroupingTracker.flushAndClean()
pc.eventsCh <- req
// wait for the request to complete
<-req.doneCh
return req.err
}
func (pc *partitionConsumer) internalSeekByTime(seek *seekByTimeRequest) {
defer close(seek.doneCh)
state := pc.getConsumerState()
if state == consumerClosing || state == consumerClosed {
pc.log.WithField("state", pc.state).Error("Failed seekByTime by consumer is closing or has closed")
return
}
requestID := pc.client.rpcClient.NewRequestID()
cmdSeek := &pb.CommandSeek{
ConsumerId: proto.Uint64(pc.consumerID),
RequestId: proto.Uint64(requestID),
MessagePublishTime: proto.Uint64(uint64(seek.publishTime.UnixNano() / int64(time.Millisecond))),
}
_, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), requestID, pb.BaseCommand_SEEK, cmdSeek)
if err != nil {
pc.log.WithError(err).Error("Failed to reset to message publish time")
seek.err = err
return
}
pc.clearReceiverQueue()
}
func (pc *partitionConsumer) internalAck(req *ackRequest) {
defer close(req.doneCh)
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer")
return
}
msgID := req.msgID
messageIDs := make([]*pb.MessageIdData, 1)
messageIDs[0] = &pb.MessageIdData{
LedgerId: proto.Uint64(uint64(msgID.ledgerID)),
EntryId: proto.Uint64(uint64(msgID.entryID)),
}
if pc.options.enableBatchIndexAck && msgID.tracker != nil {
ackSet := msgID.tracker.toAckSet()
if ackSet != nil {
messageIDs[0].AckSet = ackSet
}
}
reqID := pc.client.rpcClient.NewRequestID()
cmdAck := &pb.CommandAck{
ConsumerId: proto.Uint64(pc.consumerID),
MessageId: messageIDs,
}
switch req.ackType {
case individualAck:
cmdAck.AckType = pb.CommandAck_Individual.Enum()
case cumulativeAck:
cmdAck.AckType = pb.CommandAck_Cumulative.Enum()
}
if pc.options.ackWithResponse {
cmdAck.RequestId = proto.Uint64(reqID)
_, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), reqID, pb.BaseCommand_ACK, cmdAck)
if err != nil {
pc.log.WithError(err).Error("Ack with response error")
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")
req.err = err
}
}
func (pc *partitionConsumer) internalAckList(msgIDs []*pb.MessageIdData) {
pc.client.rpcClient.RequestOnCnxNoWait(pc._getConn(), pb.BaseCommand_ACK, &pb.CommandAck{
AckType: pb.CommandAck_Individual.Enum(),
ConsumerId: proto.Uint64(pc.consumerID),
MessageId: msgIDs,
})
}
func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, headersAndPayload internal.Buffer) error {
pbMsgID := response.GetMessageId()
reader := internal.NewMessageReader(headersAndPayload)
brokerMetadata, err := reader.ReadBrokerMetadata()
if err != nil {
// todo optimize use more appropriate error codes
pc.discardCorruptedMessage(pbMsgID, pb.CommandAck_BatchDeSerializeError)
return err
}
msgMeta, err := reader.ReadMessageMetadata()
if err != nil {
pc.discardCorruptedMessage(pbMsgID, pb.CommandAck_ChecksumMismatch)
return err
}
decryptedPayload, err := pc.decryptor.Decrypt(headersAndPayload.ReadableSlice(), pbMsgID, msgMeta)
// error decrypting the payload
if err != nil {
// default crypto failure action
crypToFailureAction := crypto.ConsumerCryptoFailureActionFail
if pc.options.decryption != nil {
crypToFailureAction = pc.options.decryption.ConsumerCryptoFailureAction
}