forked from Comcast/pulsar-client-go
-
Notifications
You must be signed in to change notification settings - Fork 0
/
managed_consumer.go
424 lines (357 loc) · 9.91 KB
/
managed_consumer.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
// Copyright 2018 Comcast Cable Communications Management, LLC
// Licensed 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 (
"context"
"errors"
"sync"
"time"
)
// ManagedConsumerConfig is used to configure a ManagedConsumer.
type ManagedConsumerConfig struct {
ManagedClientConfig
Topic string
Name string // subscription name
Exclusive bool // if false, subscription is shared
Earliest bool // if true, subscription cursor set to beginning
QueueSize int // number of messages to buffer before dropping messages
NewConsumerTimeout time.Duration // maximum duration to create Consumer, including topic lookup
InitialReconnectDelay time.Duration // how long to initially wait to reconnect Producer
MaxReconnectDelay time.Duration // maximum time to wait to attempt to reconnect Producer
}
// setDefaults returns a modified config with appropriate zero values set to defaults.
func (m ManagedConsumerConfig) setDefaults() ManagedConsumerConfig {
if m.NewConsumerTimeout <= 0 {
m.NewConsumerTimeout = 5 * time.Second
}
if m.InitialReconnectDelay <= 0 {
m.InitialReconnectDelay = 1 * time.Second
}
if m.MaxReconnectDelay <= 0 {
m.MaxReconnectDelay = 5 * time.Minute
}
// unbuffered queue not allowed
if m.QueueSize <= 0 {
m.QueueSize = 128
}
return m
}
// NewManagedConsumer returns an initialized ManagedConsumer. It will create and recreate
// a Consumer for the given discovery address and topic on a background goroutine.
func NewManagedConsumer(cp *ManagedClientPool, cfg ManagedConsumerConfig) *ManagedConsumer {
cfg = cfg.setDefaults()
m := ManagedConsumer{
clientPool: cp,
cfg: cfg,
asyncErrs: asyncErrors(cfg.Errs),
queue: make(chan Message, cfg.QueueSize),
waitc: make(chan struct{}),
}
go m.manage()
return &m
}
// ManagedConsumer wraps a Consumer with reconnect logic.
type ManagedConsumer struct {
clientPool *ManagedClientPool
cfg ManagedConsumerConfig
asyncErrs asyncErrors
queue chan Message
mu sync.RWMutex // protects following
consumer *Consumer // either consumer is nil and wait isn't or vice versa
waitc chan struct{} // if consumer is nil, this will unblock when it's been re-set
}
// Ack acquires a consumer and sends an ACK message for the given message.
func (m *ManagedConsumer) Ack(ctx context.Context, msg Message) error {
for {
m.mu.RLock()
consumer := m.consumer
wait := m.waitc
m.mu.RUnlock()
if consumer == nil {
select {
case <-wait:
// a new consumer was established.
// Re-enter read-lock to obtain it.
continue
case <-ctx.Done():
return ctx.Err()
}
}
return consumer.Ack(msg)
}
}
// Receive returns a single Message, if available.
// A reasonable context should be provided that will be used
// to wait for an incoming message if none are available.
func (m *ManagedConsumer) Receive(ctx context.Context) (Message, error) {
for {
m.mu.RLock()
consumer := m.consumer
wait := m.waitc
m.mu.RUnlock()
if consumer == nil {
select {
case <-wait:
// a new consumer was established.
// Re-enter read-lock to obtain it.
continue
case <-ctx.Done():
return Message{}, ctx.Err()
}
}
// TODO: determine when, if ever, to call
// consumer.RedeliverOverflow
if err := consumer.Flow(1); err != nil {
return Message{}, err
}
select {
case msg := <-m.queue:
return msg, nil
case <-ctx.Done():
return Message{}, ctx.Err()
case <-consumer.Closed():
return Message{}, errors.New("consumer closed")
case <-consumer.ConnClosed():
return Message{}, errors.New("consumer connection closed")
}
}
}
// ReceiveAsync blocks until the context is done. It continuously reads messages from the
// consumer and sends them to the provided channel. It manages flow control internally based
// on the queue size.
func (m *ManagedConsumer) ReceiveAsync(ctx context.Context, msgs chan<- Message) error {
// send flow request after 1/2 of the queue
// has been consumed
highwater := uint32(cap(m.queue)) / 2
drain := func() {
for {
select {
case msg := <-m.queue:
msgs <- msg
default:
return
}
}
}
CONSUMER:
for {
// ensure that the message queue is empty
drain()
// gain lock on consumer
m.mu.RLock()
consumer := m.consumer
wait := m.waitc
m.mu.RUnlock()
if consumer == nil {
select {
case <-wait:
// a new consumer was established.
// Re-enter read-lock to obtain it.
continue
case <-ctx.Done():
return ctx.Err()
}
}
// TODO: determine when, if ever, to call
// consumer.RedeliverOverflow
// request half the buffer's capacity
if err := consumer.Flow(highwater); err != nil {
m.asyncErrs.send(err)
continue CONSUMER
}
var receivedSinceFlow uint32
for {
select {
case msg := <-m.queue:
msgs <- msg
if receivedSinceFlow++; receivedSinceFlow >= highwater {
if err := consumer.Flow(receivedSinceFlow); err != nil {
m.asyncErrs.send(err)
continue CONSUMER
}
receivedSinceFlow = 0
}
continue
case <-ctx.Done():
return ctx.Err()
case <-consumer.Closed():
m.asyncErrs.send(errors.New("consumer closed"))
continue CONSUMER
case <-consumer.ConnClosed():
m.asyncErrs.send(errors.New("consumer connection closed"))
continue CONSUMER
}
}
}
}
// set unblocks the "wait" channel (if not nil),
// and sets the consumer under lock.
func (m *ManagedConsumer) set(c *Consumer) {
m.mu.Lock()
m.consumer = c
if m.waitc != nil {
close(m.waitc)
m.waitc = nil
}
m.mu.Unlock()
}
// unset creates the "wait" channel (if nil),
// and sets the consumer to nil under lock.
func (m *ManagedConsumer) unset() {
m.mu.Lock()
if m.waitc == nil {
// allow unset() to be called
// multiple times by only creating
// wait chan if its nil
m.waitc = make(chan struct{})
}
m.consumer = nil
m.mu.Unlock()
}
// newConsumer attempts to create a Consumer.
func (m *ManagedConsumer) newConsumer(ctx context.Context) (*Consumer, error) {
mc, err := m.clientPool.ForTopic(ctx, m.cfg.ManagedClientConfig, m.cfg.Topic)
if err != nil {
return nil, err
}
client, err := mc.Get(ctx)
if err != nil {
return nil, err
}
// Create the topic consumer. A non-blank consumer name is required.
if m.cfg.Exclusive {
return client.NewExclusiveConsumer(ctx, m.cfg.Topic, m.cfg.Name, m.cfg.Earliest, m.queue)
}
return client.NewSharedConsumer(ctx, m.cfg.Topic, m.cfg.Name, m.queue)
}
// reconnect blocks while a new Consumer is created.
func (m *ManagedConsumer) reconnect(initial bool) *Consumer {
retryDelay := m.cfg.InitialReconnectDelay
for attempt := 1; ; attempt++ {
if initial {
initial = false
} else {
<-time.After(retryDelay)
if retryDelay < m.cfg.MaxReconnectDelay {
// double retry delay until we reach the max
if retryDelay *= 2; retryDelay > m.cfg.MaxReconnectDelay {
retryDelay = m.cfg.MaxReconnectDelay
}
}
}
ctx, cancel := context.WithTimeout(context.Background(), m.cfg.NewConsumerTimeout)
newConsumer, err := m.newConsumer(ctx)
cancel()
if err != nil {
m.asyncErrs.send(err)
continue
}
return newConsumer
}
}
// manage Monitors the Consumer for conditions
// that require it to be recreated.
func (m *ManagedConsumer) manage() {
defer m.unset()
consumer := m.reconnect(true)
m.set(consumer)
for {
select {
case <-consumer.ReachedEndOfTopic():
// TODO: What to do here? For now, reconnect
// reconnect
case <-consumer.Closed():
// reconnect
case <-consumer.ConnClosed():
// reconnect
}
m.unset()
consumer = m.reconnect(false)
m.set(consumer)
}
}
// RedeliverUnacknowledged sends of REDELIVER_UNACKNOWLEDGED_MESSAGES request
// for all messages that have not been acked.
func (m *ManagedConsumer) RedeliverUnacknowledged(ctx context.Context) error {
for {
m.mu.RLock()
consumer := m.consumer
wait := m.waitc
m.mu.RUnlock()
if consumer == nil {
select {
case <-wait:
// a new consumer was established.
// Re-enter read-lock to obtain it.
continue
case <-ctx.Done():
return ctx.Err()
}
}
return consumer.RedeliverUnacknowledged(ctx)
}
}
// RedeliverOverflow sends of REDELIVER_UNACKNOWLEDGED_MESSAGES request
// for all messages that were dropped because of full message buffer. Note that
// for all subscription types other than `shared`, _all_ unacknowledged messages
// will be redelivered.
// https://github.com/apache/incubator-pulsar/issues/2003
func (m *ManagedConsumer) RedeliverOverflow(ctx context.Context) (int, error) {
for {
m.mu.RLock()
consumer := m.consumer
wait := m.waitc
m.mu.RUnlock()
if consumer == nil {
select {
case <-wait:
// a new consumer was established.
// Re-enter read-lock to obtain it.
continue
case <-ctx.Done():
return -1, ctx.Err()
}
}
return consumer.RedeliverOverflow(ctx)
}
}
// Unsubscribe the consumer from its topic.
func (m *ManagedConsumer) Unsubscribe(ctx context.Context) error {
for {
m.mu.RLock()
consumer := m.consumer
wait := m.waitc
m.mu.RUnlock()
if consumer == nil {
select {
case <-wait:
// a new consumer was established.
// Re-enter read-lock to obtain it.
continue
case <-ctx.Done():
return ctx.Err()
}
}
return consumer.Unsubscribe(ctx)
}
}
// Monitor a scoped deferrable lock
func (m *ManagedConsumer) Monitor() func() {
m.mu.Lock()
return m.mu.Unlock
}
// Close consumer
func (m *ManagedConsumer) Close(ctx context.Context) error {
defer m.Monitor()()
return m.consumer.Close(ctx)
}