forked from apache/pulsar-client-go
-
Notifications
You must be signed in to change notification settings - Fork 1
/
Copy pathconsumer_partition.go
1065 lines (904 loc) · 29.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 (
"fmt"
"math"
"sync"
"time"
"github.com/gogo/protobuf/proto"
"github.com/apache/pulsar-client-go/pulsar/internal"
"github.com/apache/pulsar-client-go/pulsar/internal/compression"
pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto"
"github.com/apache/pulsar-client-go/pulsar/log"
)
var (
lastestMessageID = LatestMessageID()
)
type consumerState int
const (
consumerInit consumerState = iota
consumerReady
consumerClosing
consumerClosed
)
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 (
noMessageEntry = -1
)
type partitionConsumerOpts struct {
topic string
consumerName string
subscription string
subscriptionType SubscriptionType
subscriptionInitPos SubscriptionInitialPosition
partitionIdx int
receiverQueueSize int
nackRedeliveryDelay time.Duration
metadata map[string]string
replicateSubscriptionState bool
startMessageID trackingMessageID
startMessageIDInclusive bool
subscriptionMode subscriptionMode
readCompacted bool
disableForceTopicCreation bool
interceptors ConsumerInterceptors
maxReconnectToBroker *uint
keySharedPolicy *KeySharedPolicy
schema Schema
}
type partitionConsumer struct {
client *client
// this is needed for sending ConsumerMessage on the messageCh
parentConsumer Consumer
state consumerState
options *partitionConsumerOpts
conn internal.Connection
topic string
name string
consumerID uint64
partitionIdx int32
// shared channel
messageCh chan ConsumerMessage
// the number of message slots available
availablePermits int32
// the size of the queue channel for buffering messages
queueSize int32
queueCh chan []*message
startMessageID trackingMessageID
lastDequeuedMsg trackingMessageID
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 map[pb.CompressionType]compression.Provider
metrics *internal.TopicMetrics
}
func newPartitionConsumer(parent Consumer, client *client, options *partitionConsumerOpts,
messageCh chan ConsumerMessage, dlq *dlqRouter,
metrics *internal.TopicMetrics) (*partitionConsumer, error) {
pc := &partitionConsumer{
state: consumerInit,
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),
queueSize: int32(options.receiverQueueSize),
queueCh: make(chan []*message, options.receiverQueueSize),
startMessageID: options.startMessageID,
connectedCh: make(chan struct{}),
messageCh: messageCh,
connectClosedCh: make(chan connectionClosed, 10),
closeCh: make(chan struct{}),
clearQueueCh: make(chan func(id trackingMessageID)),
compressionProviders: make(map[pb.CompressionType]compression.Provider),
dlq: dlq,
metrics: metrics,
}
pc.log = client.log.SubLogger(log.Fields{
"name": pc.name,
"topic": options.topic,
"subscription": options.subscription,
"consumerID": pc.consumerID,
})
pc.nackTracker = newNegativeAcksTracker(pc, options.nackRedeliveryDelay, pc.log)
err := pc.grabConn()
if err != nil {
pc.log.WithError(err).Error("Failed to create consumer")
pc.nackTracker.Close()
return nil, err
}
pc.log.Info("Created consumer")
pc.state = consumerReady
if pc.options.startMessageIDInclusive && pc.startMessageID == lastestMessageID {
msgID, err := pc.requestGetLastMessageID()
if err != nil {
pc.nackTracker.Close()
return nil, err
}
if msgID.entryID != noMessageEntry {
pc.startMessageID = msgID
err = pc.requestSeek(msgID.messageID)
if err != nil {
pc.nackTracker.Close()
return nil, err
}
}
}
go pc.dispatcher()
go pc.runEventsLoop()
return pc, nil
}
func (pc *partitionConsumer) Unsubscribe() error {
req := &unsubscribeRequest{doneCh: make(chan struct{})}
pc.eventsCh <- req
// wait for the request to complete
<-req.doneCh
return req.err
}
func (pc *partitionConsumer) internalUnsubscribe(unsub *unsubscribeRequest) {
defer close(unsub.doneCh)
if pc.state == consumerClosed || pc.state == consumerClosing {
pc.log.Error("Failed to unsubscribe consumer, the consumer is closing or consumer has been closed")
return
}
pc.state = consumerClosing
requestID := pc.client.rpcClient.NewRequestID()
cmdUnsubscribe := &pb.CommandUnsubscribe{
RequestId: proto.Uint64(requestID),
ConsumerId: proto.Uint64(pc.consumerID),
}
_, err := pc.client.rpcClient.RequestOnCnx(pc.conn, 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.state = consumerReady
// Should'nt remove the consumer handler
return
}
pc.conn.DeleteConsumeHandler(pc.consumerID)
if pc.nackTracker != nil {
pc.nackTracker.Close()
}
pc.log.Infof("The consumer[%d] successfully unsubscribed", pc.consumerID)
pc.state = consumerClosed
}
func (pc *partitionConsumer) getLastMessageID() (trackingMessageID, error) {
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) {
requestID := pc.client.rpcClient.NewRequestID()
cmdGetLastMessageID := &pb.CommandGetLastMessageId{
RequestId: proto.Uint64(requestID),
ConsumerId: proto.Uint64(pc.consumerID),
}
res, err := pc.client.rpcClient.RequestOnCnx(pc.conn, requestID,
pb.BaseCommand_GET_LAST_MESSAGE_ID, cmdGetLastMessageID)
if err != nil {
pc.log.WithError(err).Error("Failed to get last message id")
return trackingMessageID{}, err
}
id := res.Response.GetLastMessageIdResponse.GetLastMessageId()
return convertToMessageID(id), nil
}
func (pc *partitionConsumer) AckID(msgID trackingMessageID) {
if !msgID.Undefined() && msgID.ack() {
pc.metrics.AcksCounter.Inc()
pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-msgID.receivedTime.UnixNano()) / 1.0e9)
req := &ackRequest{
msgID: msgID,
}
pc.eventsCh <- req
pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID)
}
}
func (pc *partitionConsumer) NackID(msgID trackingMessageID) {
pc.nackTracker.Add(msgID.messageID)
pc.metrics.NacksCounter.Inc()
}
func (pc *partitionConsumer) Redeliver(msgIds []messageID) {
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) {
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)),
}
}
pc.client.rpcClient.RequestOnCnxNoWait(pc.conn,
pb.BaseCommand_REDELIVER_UNACKNOWLEDGED_MESSAGES, &pb.CommandRedeliverUnacknowledgedMessages{
ConsumerId: proto.Uint64(pc.consumerID),
MessageIds: msgIDDataList,
})
}
func (pc *partitionConsumer) Close() {
if pc.state != consumerReady {
return
}
req := &closeRequest{doneCh: make(chan struct{})}
pc.eventsCh <- req
// wait for request to finish
<-req.doneCh
}
func (pc *partitionConsumer) Seek(msgID trackingMessageID) error {
req := &seekRequest{
doneCh: make(chan struct{}),
msgID: msgID,
}
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.messageID)
}
func (pc *partitionConsumer) requestSeek(msgID messageID) error {
if pc.state == consumerClosing || pc.state == consumerClosed {
pc.log.Error("Consumer was already 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.conn, 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 {
req := &seekByTimeRequest{
doneCh: make(chan struct{}),
publishTime: time,
}
pc.eventsCh <- req
// wait for the request to complete
<-req.doneCh
return req.err
}
func (pc *partitionConsumer) internalSeekByTime(seek *seekByTimeRequest) {
defer close(seek.doneCh)
if pc.state == consumerClosing || pc.state == consumerClosed {
pc.log.Error("Consumer was already 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.conn, requestID, pb.BaseCommand_SEEK, cmdSeek)
if err != nil {
pc.log.WithError(err).Error("Failed to reset to message publish time")
seek.err = err
}
}
func (pc *partitionConsumer) internalAck(req *ackRequest) {
msgID := req.msgID
messageIDs := make([]*pb.MessageIdData, 1)
messageIDs[0] = &pb.MessageIdData{
LedgerId: proto.Uint64(uint64(msgID.ledgerID)),
EntryId: proto.Uint64(uint64(msgID.entryID)),
}
cmdAck := &pb.CommandAck{
ConsumerId: proto.Uint64(pc.consumerID),
MessageId: messageIDs,
AckType: pb.CommandAck_Individual.Enum(),
}
pc.client.rpcClient.RequestOnCnxNoWait(pc.conn, pb.BaseCommand_ACK, cmdAck)
}
func (pc *partitionConsumer) MessageReceived(response *pb.CommandMessage, headersAndPayload internal.Buffer) error {
pbMsgID := response.GetMessageId()
reader := internal.NewMessageReader(headersAndPayload)
msgMeta, err := reader.ReadMessageMetadata()
if err != nil {
pc.discardCorruptedMessage(pbMsgID, pb.CommandAck_ChecksumMismatch)
return err
}
uncompressedHeadersAndPayload, err := pc.Decompress(msgMeta, headersAndPayload)
if err != nil {
pc.discardCorruptedMessage(pbMsgID, pb.CommandAck_DecompressionError)
return err
}
// Reset the reader on the uncompressed buffer
reader.ResetBuffer(uncompressedHeadersAndPayload)
numMsgs := 1
if msgMeta.NumMessagesInBatch != nil {
numMsgs = int(msgMeta.GetNumMessagesInBatch())
}
messages := make([]*message, 0)
var ackTracker *ackTracker
// are there multiple messages in this batch?
if numMsgs > 1 {
ackTracker = newAckTracker(numMsgs)
}
pc.metrics.MessagesReceived.Add(float64(numMsgs))
pc.metrics.PrefetchedMessages.Add(float64(numMsgs))
for i := 0; i < numMsgs; i++ {
smm, payload, err := reader.ReadMessage()
if err != nil {
pc.discardCorruptedMessage(pbMsgID, pb.CommandAck_BatchDeSerializeError)
return err
}
pc.metrics.BytesReceived.Add(float64(len(payload)))
pc.metrics.PrefetchedBytes.Add(float64(len(payload)))
msgID := newTrackingMessageID(
int64(pbMsgID.GetLedgerId()),
int64(pbMsgID.GetEntryId()),
int32(i),
pc.partitionIdx,
ackTracker)
if pc.messageShouldBeDiscarded(msgID) {
pc.AckID(msgID)
continue
}
// set the consumer so we know how to ack the message id
msgID.consumer = pc
var msg *message
if smm != nil {
msg = &message{
publishTime: timeFromUnixTimestampMillis(msgMeta.GetPublishTime()),
eventTime: timeFromUnixTimestampMillis(smm.GetEventTime()),
key: smm.GetPartitionKey(),
producerName: msgMeta.GetProducerName(),
properties: internal.ConvertToStringMap(smm.GetProperties()),
topic: pc.topic,
msgID: msgID,
payLoad: payload,
schema: pc.options.schema,
replicationClusters: msgMeta.GetReplicateTo(),
replicatedFrom: msgMeta.GetReplicatedFrom(),
redeliveryCount: response.GetRedeliveryCount(),
}
} else {
msg = &message{
publishTime: timeFromUnixTimestampMillis(msgMeta.GetPublishTime()),
eventTime: timeFromUnixTimestampMillis(msgMeta.GetEventTime()),
key: msgMeta.GetPartitionKey(),
producerName: msgMeta.GetProducerName(),
properties: internal.ConvertToStringMap(msgMeta.GetProperties()),
topic: pc.topic,
msgID: msgID,
payLoad: payload,
schema: pc.options.schema,
replicationClusters: msgMeta.GetReplicateTo(),
replicatedFrom: msgMeta.GetReplicatedFrom(),
redeliveryCount: response.GetRedeliveryCount(),
}
}
pc.options.interceptors.BeforeConsume(ConsumerMessage{
Consumer: pc.parentConsumer,
Message: msg,
})
messages = append(messages, msg)
}
// send messages to the dispatcher
pc.queueCh <- messages
return nil
}
func (pc *partitionConsumer) messageShouldBeDiscarded(msgID trackingMessageID) bool {
if pc.startMessageID.Undefined() {
return false
}
if pc.options.startMessageIDInclusive {
return pc.startMessageID.greater(msgID.messageID)
}
// Non inclusive
return pc.startMessageID.greaterEqual(msgID.messageID)
}
func (pc *partitionConsumer) ConnectionClosed() {
// Trigger reconnection in the consumer goroutine
pc.log.Debug("connection closed and send to connectClosedCh")
pc.connectClosedCh <- connectionClosed{}
}
// Flow command gives additional permits to send messages to the consumer.
// A typical consumer implementation will use a queue to accumulate these messages
// before the application is ready to consume them. After the consumer is ready,
// the client needs to give permission to the broker to push messages.
func (pc *partitionConsumer) internalFlow(permits uint32) error {
if permits == 0 {
return fmt.Errorf("invalid number of permits requested: %d", permits)
}
cmdFlow := &pb.CommandFlow{
ConsumerId: proto.Uint64(pc.consumerID),
MessagePermits: proto.Uint32(permits),
}
pc.client.rpcClient.RequestOnCnxNoWait(pc.conn, pb.BaseCommand_FLOW, cmdFlow)
return nil
}
// dispatcher manages the internal message queue channel
// and manages the flow control
func (pc *partitionConsumer) dispatcher() {
defer func() {
pc.log.Debug("exiting dispatch loop")
}()
var messages []*message
for {
var queueCh chan []*message
var messageCh chan ConsumerMessage
var nextMessage ConsumerMessage
// are there more messages to send?
if len(messages) > 0 {
nextMessage = ConsumerMessage{
Consumer: pc.parentConsumer,
Message: messages[0],
}
if pc.dlq.shouldSendToDlq(&nextMessage) {
// pass the message to the DLQ router
pc.metrics.DlqCounter.Inc()
messageCh = pc.dlq.Chan()
} else {
// pass the message to application channel
messageCh = pc.messageCh
}
pc.metrics.PrefetchedMessages.Dec()
pc.metrics.PrefetchedBytes.Sub(float64(len(messages[0].payLoad)))
} else {
// we are ready for more messages
queueCh = pc.queueCh
}
select {
case <-pc.closeCh:
return
case _, ok := <-pc.connectedCh:
if !ok {
return
}
pc.log.Debug("dispatcher received connection event")
messages = nil
// reset available permits
pc.availablePermits = 0
initialPermits := uint32(pc.queueSize)
pc.log.Debugf("dispatcher requesting initial permits=%d", initialPermits)
// send initial permits
if err := pc.internalFlow(initialPermits); err != nil {
pc.log.WithError(err).Error("unable to send initial permits to broker")
}
case msgs, ok := <-queueCh:
if !ok {
return
}
// we only read messages here after the consumer has processed all messages
// in the previous batch
messages = msgs
// if the messageCh is nil or the messageCh is full this will not be selected
case messageCh <- nextMessage:
// allow this message to be garbage collected
messages[0] = nil
messages = messages[1:]
// TODO implement a better flow controller
// send more permits if needed
pc.availablePermits++
flowThreshold := int32(math.Max(float64(pc.queueSize/2), 1))
if pc.availablePermits >= flowThreshold {
availablePermits := pc.availablePermits
requestedPermits := availablePermits
pc.availablePermits = 0
pc.log.Debugf("requesting more permits=%d available=%d", requestedPermits, availablePermits)
if err := pc.internalFlow(uint32(requestedPermits)); err != nil {
pc.log.WithError(err).Error("unable to send permits")
}
}
case clearQueueCb := <-pc.clearQueueCh:
// drain the message queue on any new connection by sending a
// special nil message to the channel so we know when to stop dropping messages
var nextMessageInQueue trackingMessageID
go func() {
pc.queueCh <- nil
}()
for m := range pc.queueCh {
// the queue has been drained
if m == nil {
break
} else if nextMessageInQueue.Undefined() {
nextMessageInQueue = m[0].msgID.(trackingMessageID)
}
}
clearQueueCb(nextMessageInQueue)
}
}
}
type ackRequest struct {
msgID trackingMessageID
}
type unsubscribeRequest struct {
doneCh chan struct{}
err error
}
type closeRequest struct {
doneCh chan struct{}
}
type redeliveryRequest struct {
msgIds []messageID
}
type getLastMsgIDRequest struct {
doneCh chan struct{}
msgID trackingMessageID
err error
}
type seekRequest struct {
doneCh chan struct{}
msgID trackingMessageID
err error
}
type seekByTimeRequest struct {
doneCh chan struct{}
publishTime time.Time
err error
}
func (pc *partitionConsumer) runEventsLoop() {
defer func() {
pc.log.Debug("exiting events loop")
}()
pc.log.Debug("get into runEventsLoop")
go func() {
for {
select {
case <-pc.closeCh:
return
case <-pc.connectClosedCh:
pc.log.Debug("runEventsLoop will reconnect")
pc.reconnectToBroker()
}
}
}()
for {
for i := range pc.eventsCh {
switch v := i.(type) {
case *ackRequest:
pc.internalAck(v)
case *redeliveryRequest:
pc.internalRedeliver(v)
case *unsubscribeRequest:
pc.internalUnsubscribe(v)
case *getLastMsgIDRequest:
pc.internalGetLastMessageID(v)
case *seekRequest:
pc.internalSeek(v)
case *seekByTimeRequest:
pc.internalSeekByTime(v)
case *closeRequest:
pc.internalClose(v)
return
}
}
}
}
func (pc *partitionConsumer) internalClose(req *closeRequest) {
defer close(req.doneCh)
if pc.state != consumerReady {
// this might be redundant but to ensure nack tracker is closed
if pc.nackTracker != nil {
pc.nackTracker.Close()
}
return
}
if pc.state == consumerClosed || pc.state == consumerClosing {
pc.log.Error("The consumer is closing or has been closed")
if pc.nackTracker != nil {
pc.nackTracker.Close()
}
return
}
pc.state = consumerClosing
pc.log.Infof("Closing consumer=%d", pc.consumerID)
requestID := pc.client.rpcClient.NewRequestID()
cmdClose := &pb.CommandCloseConsumer{
ConsumerId: proto.Uint64(pc.consumerID),
RequestId: proto.Uint64(requestID),
}
_, err := pc.client.rpcClient.RequestOnCnx(pc.conn, requestID, pb.BaseCommand_CLOSE_CONSUMER, cmdClose)
if err != nil {
pc.log.WithError(err).Warn("Failed to close consumer")
} else {
pc.log.Info("Closed consumer")
}
for _, provider := range pc.compressionProviders {
provider.Close()
}
pc.state = consumerClosed
pc.conn.DeleteConsumeHandler(pc.consumerID)
if pc.nackTracker != nil {
pc.nackTracker.Close()
}
close(pc.closeCh)
}
func (pc *partitionConsumer) reconnectToBroker() {
var (
maxRetry int
backoff = internal.Backoff{}
)
if pc.options.maxReconnectToBroker == nil {
maxRetry = -1
} else {
maxRetry = int(*pc.options.maxReconnectToBroker)
}
for maxRetry != 0 {
if pc.state != consumerReady {
// Consumer is already closing
return
}
d := backoff.Next()
pc.log.Info("Reconnecting to broker in ", d)
time.Sleep(d)
err := pc.grabConn()
if err == nil {
// Successfully reconnected
pc.log.Info("Reconnected consumer to broker")
return
}
if maxRetry > 0 {
maxRetry--
}
}
}
func (pc *partitionConsumer) grabConn() error {
lr, err := pc.client.lookupService.Lookup(pc.topic)
if err != nil {
pc.log.WithError(err).Warn("Failed to lookup topic")
return err
}
pc.log.Debugf("Lookup result: %+v", lr)
subType := toProtoSubType(pc.options.subscriptionType)
initialPosition := toProtoInitialPosition(pc.options.subscriptionInitPos)
keySharedMeta := toProtoKeySharedMeta(pc.options.keySharedPolicy)
requestID := pc.client.rpcClient.NewRequestID()
pbSchema := new(pb.Schema)
if pc.options.schema != nil && pc.options.schema.GetSchemaInfo() != nil {
tmpSchemaType := pb.Schema_Type(int32(pc.options.schema.GetSchemaInfo().Type))
pbSchema = &pb.Schema{
Name: proto.String(pc.options.schema.GetSchemaInfo().Name),
Type: &tmpSchemaType,
SchemaData: []byte(pc.options.schema.GetSchemaInfo().Schema),
Properties: internal.ConvertFromStringMap(pc.options.schema.GetSchemaInfo().Properties),
}
pc.log.Debugf("The partition consumer schema name is: %s", pbSchema.Name)
} else {
pbSchema = nil
pc.log.Debug("The partition consumer schema is nil")
}
cmdSubscribe := &pb.CommandSubscribe{
Topic: proto.String(pc.topic),
Subscription: proto.String(pc.options.subscription),
SubType: subType.Enum(),
ConsumerId: proto.Uint64(pc.consumerID),
RequestId: proto.Uint64(requestID),
ConsumerName: proto.String(pc.name),
PriorityLevel: nil,
Durable: proto.Bool(pc.options.subscriptionMode == durable),
Metadata: internal.ConvertFromStringMap(pc.options.metadata),
ReadCompacted: proto.Bool(pc.options.readCompacted),
Schema: pbSchema,
InitialPosition: initialPosition.Enum(),
ReplicateSubscriptionState: proto.Bool(pc.options.replicateSubscriptionState),
KeySharedMeta: keySharedMeta,
}
pc.startMessageID = pc.clearReceiverQueue()
if pc.options.subscriptionMode != durable {
// For regular subscriptions the broker will determine the restarting point
cmdSubscribe.StartMessageId = convertToMessageIDData(pc.startMessageID)
}
if len(pc.options.metadata) > 0 {
cmdSubscribe.Metadata = toKeyValues(pc.options.metadata)
}
// force topic creation is enabled by default so
// we only need to set the flag when disabling it
if pc.options.disableForceTopicCreation {
cmdSubscribe.ForceTopicCreation = proto.Bool(false)
}
res, err := pc.client.rpcClient.Request(lr.LogicalAddr, lr.PhysicalAddr, requestID,
pb.BaseCommand_SUBSCRIBE, cmdSubscribe)
if err != nil {
pc.log.WithError(err).Error("Failed to create consumer")
return err
}
if res.Response.ConsumerStatsResponse != nil {
pc.name = res.Response.ConsumerStatsResponse.GetConsumerName()
}
pc.conn = res.Cnx
pc.log.Info("Connected consumer")
pc.conn.AddConsumeHandler(pc.consumerID, pc)
msgType := res.Response.GetType()
switch msgType {
case pb.BaseCommand_SUCCESS:
// notify the dispatcher we have connection
go func() {
pc.connectedCh <- struct{}{}
}()
return nil
case pb.BaseCommand_ERROR:
errMsg := res.Response.GetError()
return fmt.Errorf("%s: %s", errMsg.GetError().String(), errMsg.GetMessage())
default:
return newUnexpectedErrMsg(msgType, requestID)
}
}
func (pc *partitionConsumer) clearQueueAndGetNextMessage() trackingMessageID {
if pc.state != consumerReady {
return trackingMessageID{}
}
wg := &sync.WaitGroup{}
wg.Add(1)
var msgID trackingMessageID
pc.clearQueueCh <- func(id trackingMessageID) {
msgID = id
wg.Done()
}
wg.Wait()
return msgID
}
/**
* Clear the internal receiver queue and returns the message id of what was the 1st message in the queue that was
* not seen by the application
*/
func (pc *partitionConsumer) clearReceiverQueue() trackingMessageID {
nextMessageInQueue := pc.clearQueueAndGetNextMessage()
if pc.startMessageID.Undefined() {
return pc.startMessageID
}
if !nextMessageInQueue.Undefined() {
return getPreviousMessage(nextMessageInQueue)
} else if !pc.lastDequeuedMsg.Undefined() {
// If the queue was empty we need to restart from the message just after the last one that has been dequeued
// in the past
return pc.lastDequeuedMsg
} else {
// No message was received or dequeued by this consumer. Next message would still be the startMessageId
return pc.startMessageID
}
}
func getPreviousMessage(mid trackingMessageID) trackingMessageID {
if mid.batchIdx >= 0 {
return trackingMessageID{
messageID: messageID{
ledgerID: mid.ledgerID,
entryID: mid.entryID,
batchIdx: mid.batchIdx - 1,
partitionIdx: mid.partitionIdx,
},
tracker: mid.tracker,
consumer: mid.consumer,
receivedTime: mid.receivedTime,
}
}
// Get on previous message in previous entry
return trackingMessageID{
messageID: messageID{
ledgerID: mid.ledgerID,
entryID: mid.entryID - 1,
batchIdx: mid.batchIdx,
partitionIdx: mid.partitionIdx,
},
tracker: mid.tracker,
consumer: mid.consumer,
receivedTime: mid.receivedTime,
}
}
func (pc *partitionConsumer) Decompress(msgMeta *pb.MessageMetadata, payload internal.Buffer) (internal.Buffer, error) {
provider, ok := pc.compressionProviders[msgMeta.GetCompression()]
if !ok {
var err error
if provider, err = pc.initializeCompressionProvider(msgMeta.GetCompression()); err != nil {
pc.log.WithError(err).Error("Failed to decompress message.")
return nil, err
}
pc.compressionProviders[msgMeta.GetCompression()] = provider
}
uncompressed, err := provider.Decompress(nil, payload.ReadableSlice(), int(msgMeta.GetUncompressedSize()))
if err != nil {
return nil, err