-
Notifications
You must be signed in to change notification settings - Fork 3
/
subscriber.go
362 lines (334 loc) · 11.7 KB
/
subscriber.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
package pubsub
import (
"context"
"fmt"
"github.com/paust-team/pirius/agent/storage"
"github.com/paust-team/pirius/bootstrapping"
"github.com/paust-team/pirius/bootstrapping/topic"
"github.com/paust-team/pirius/constants"
"github.com/paust-team/pirius/helper"
"github.com/paust-team/pirius/logger"
"github.com/paust-team/pirius/proto/pb"
"github.com/paust-team/pirius/qerror"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"io"
"runtime"
"sync"
"time"
)
type SubscriptionResult struct {
FragmentId uint
SeqNum uint64
Data []byte
}
type SubscriptionAddrs map[string][]uint
type subscriberBase struct {
id string
bootstrapper *bootstrapping.BootstrapService
lastSubscribedOffset storage.TopicFragmentOffsets // last fetched offsets
currentSubscriptions []uint
}
func (s subscriberBase) prepare(ctx context.Context, topicName string) (chan topic.SubscriptionInfo, []uint, error) {
var subscriptions []uint
subscriptionWatchCh, err := s.bootstrapper.WatchSubscriptionChanged(ctx, topicName)
if err != nil {
return nil, nil, err
}
logger.Info("watcher for subscriptions registered")
// register subscriber path and wait for initial rebalance
err = s.bootstrapper.AddSubscriber(topicName, s.id)
if _, ok := err.(qerror.CoordTargetAlreadyExistsError); ok { // if already registered, check subscription info
subscriptionFrame, err := s.bootstrapper.GetTopicSubscriptions(topicName)
if err != nil {
return nil, nil, err
}
initialSubscriptions := subscriptionFrame.SubscriptionInfo()
if _, ok = initialSubscriptions[s.id]; !ok {
return nil, nil, qerror.InvalidStateError{State: fmt.Sprintf("fragment not exists for subscriber(%s)", s.id)}
} else if len(initialSubscriptions[s.id]) == 0 {
return nil, nil, qerror.InvalidStateError{State: fmt.Sprintf("fragment not exists for subscriber(%s)", s.id)}
}
subscriptions = initialSubscriptions[s.id]
} else if err != nil {
return nil, nil, err
} else { // wait watch event for initial subscription assignment
timer := time.After(time.Second * constants.InitialRebalanceTimeout)
for subscriptions == nil {
select {
case initialSubscriptions := <-subscriptionWatchCh:
if _, ok := initialSubscriptions[s.id]; !ok {
continue
} else if len(initialSubscriptions[s.id]) == 0 {
continue
}
subscriptions = initialSubscriptions[s.id]
case <-timer:
return nil, nil, qerror.InvalidStateError{State: fmt.Sprintf("initial rebalance timed out for topic(%s)", topicName)}
}
}
}
return subscriptionWatchCh, subscriptions, nil
}
// helper functions
func (s subscriberBase) findSubscriptionEndpoints(topicName string, fragmentIds []uint) (SubscriptionAddrs, error) {
topicFragmentFrame, err := s.bootstrapper.GetTopicFragments(topicName)
if err != nil {
return nil, err
}
fragMappings := topicFragmentFrame.FragMappingInfo()
endpoints := make(SubscriptionAddrs)
for _, fragmentId := range fragmentIds {
if fragInfo, ok := fragMappings[fragmentId]; ok && fragInfo.State == topic.Active {
if _, ok := endpoints[fragInfo.Address]; ok {
endpoints[fragInfo.Address] = append(endpoints[fragInfo.Address], fragmentId)
} else {
endpoints[fragInfo.Address] = []uint{fragmentId}
}
}
}
return endpoints, nil
}
func (s subscriberBase) isSubscriptionUpdated(new topic.SubscriptionInfo) bool {
newSubscription, ok := new[s.id]
if !ok {
return true
}
if len(s.currentSubscriptions) != len(newSubscription) ||
!helper.HasAllElements(newSubscription, s.currentSubscriptions) {
return true
}
return false
}
type Subscriber struct {
subscriberBase
wg sync.WaitGroup
}
func NewSubscriber(id string, bootstrapper *bootstrapping.BootstrapService, subscribedOffsets storage.TopicFragmentOffsets) Subscriber {
return Subscriber{
subscriberBase: subscriberBase{
id: id,
bootstrapper: bootstrapper,
lastSubscribedOffset: subscribedOffsets,
},
wg: sync.WaitGroup{},
}
}
func (s *Subscriber) StartTopicSubscription(ctx context.Context, topicName string, batchSize, flushInterval uint32) (chan []SubscriptionResult, chan error, error) {
// register watcher for subscription info
watcherCtx, cancel := context.WithCancel(ctx)
subscriptionWatchCh, subscriptions, err := s.prepare(watcherCtx, topicName)
if err != nil {
cancel()
return nil, nil, err
}
subscriptionCtx, subscriptionCtxCancel := context.WithCancel(ctx)
subscriptionWg := sync.WaitGroup{}
subscriptionCh, sErrCh, err := s.startSubscriptions(subscriptionCtx, &subscriptionWg, topicName, subscriptions, batchSize, flushInterval)
if err != nil {
cancel()
subscriptionCtxCancel()
return nil, nil, err
}
s.currentSubscriptions = subscriptions
outStream := make(chan []SubscriptionResult)
errStream := make(chan error, 2)
s.wg.Add(1)
go func() {
defer s.wg.Done()
defer cancel()
defer subscriptionCtxCancel()
defer close(outStream)
defer close(errStream)
for {
select {
case <-ctx.Done():
logger.Info("stop subscribing: ctx.Done()", zap.String("subscriber-id", s.id))
return
case result, ok := <-subscriptionCh:
if !ok {
logger.Info("stop subscribing: receive buffer closed", zap.String("subscriber-id", s.id))
return
}
select {
case <-ctx.Done():
return
default:
outStream <- result
}
case err = <-sErrCh:
if err != nil {
errStream <- err
}
case subscriptionInfo, ok := <-subscriptionWatchCh:
if !ok {
logger.Error("stop subscribing: watch closed", zap.String("subscriber-id", s.id))
errStream <- qerror.InvalidStateError{State: "watcher channel closed unexpectedly"}
return
}
logger.Info("received new subscription info", zap.String("subscriber-id", s.id), zap.String("topic", topicName))
if s.isSubscriptionUpdated(subscriptionInfo) {
logger.Info("resetting subscriptions",
zap.String("subscriber-id", s.id),
zap.Uints("old-fragments", s.currentSubscriptions),
zap.Uints("new-fragments", subscriptionInfo[s.id]))
subscriptionCtxCancel()
subscriptionWg.Wait()
subscriptionCtx, subscriptionCtxCancel = context.WithCancel(ctx)
subscriptions = subscriptionInfo[s.id]
if len(subscriptions) == 0 { // wait for new subscription
logger.Info("received empty subscriptions. wait for new subscription",
zap.String("subscriber-id", s.id),
zap.Uints("old-fragments", s.currentSubscriptions))
} else {
subscriptionCh, sErrCh, err = s.startSubscriptions(subscriptionCtx, &subscriptionWg, topicName, subscriptions, batchSize, flushInterval)
if err != nil {
errStream <- err
return
}
logger.Info("succeed to reset subscriptions",
zap.String("subscriber-id", s.id),
zap.Uints("old-fragments", s.currentSubscriptions),
zap.Uints("new-fragments", subscriptionInfo[s.id]))
}
s.currentSubscriptions = subscriptions
} else {
logger.Info("skip: not newly subscriptions",
zap.String("subscriber-id", s.id),
zap.Uints("current-fragments", s.currentSubscriptions),
zap.Uints("received-fragments", subscriptionInfo[s.id]))
}
}
}
}()
return outStream, errStream, nil
}
func (s *Subscriber) startSubscriptions(ctx context.Context, subscriptionWg *sync.WaitGroup, topicName string, subscriptionFragments []uint,
batchSize, flushInterval uint32) (chan []SubscriptionResult, chan error, error) {
logger.Info("setup subscription streams", zap.String("subscriber-id", s.id), zap.String("topic", topicName), zap.Uints("fragmentIds", subscriptionFragments))
endpointMap, err := s.findSubscriptionEndpoints(topicName, subscriptionFragments)
if err != nil {
return nil, nil, err
}
if len(endpointMap) == 0 {
return nil, nil, qerror.TargetNotExistError{Target: fmt.Sprintf("publishers of topic '%s', fragments %v", topicName, s.currentSubscriptions)}
}
outStream := make(chan []SubscriptionResult)
errStream := make(chan error)
// create subscription stream for each endpoint
wg := sync.WaitGroup{}
for endpoint, fragmentIds := range endpointMap {
opts := grpc.WithInsecure()
conn, err := grpc.Dial(endpoint, opts)
if err != nil {
return nil, nil, err
}
// load last offsets
var subscriptionOffsets []*pb.Subscription_FragmentOffset
for _, fragmentId := range fragmentIds {
value, _ := s.lastSubscribedOffset.LoadOrStore(storage.NewFragmentKey(topicName, fragmentId), uint64(0))
lastFetchedOffset := value.(uint64)
startOffset := lastFetchedOffset + 1
subscriptionOffsets = append(subscriptionOffsets, &pb.Subscription_FragmentOffset{
FragmentId: uint32(fragmentId),
StartOffset: &startOffset,
})
}
// start gRPC stream
publisher := pb.NewPubSubClient(conn)
stream, err := publisher.Subscribe(ctx, &pb.Subscription{
Magic: 1,
TopicName: topicName,
Offsets: subscriptionOffsets,
MaxBatchSize: batchSize,
FlushInterval: flushInterval,
})
if err != nil {
stream.CloseSend()
conn.Close()
return nil, nil, err
}
wg.Add(1)
go func(pubEndpoint string) {
defer wg.Done()
defer conn.Close()
defer stream.CloseSend()
for {
select {
case <-ctx.Done():
logger.Info("stop subscribe from ctx.Done()",
zap.String("subscriber-id", s.id),
zap.String("topic", topicName),
zap.String("publisher-endpoint", pubEndpoint))
return
default:
subscriptionResult, err := stream.Recv()
if err != nil {
if err == io.EOF {
// TODO :: this is abnormal case. should be restarted?
logger.Info("stop subscribe from io.EOF",
zap.String("subscriber-id", s.id),
zap.String("topic", topicName),
zap.String("publisher-endpoint", pubEndpoint))
} else if status.Code(err) == codes.Canceled { // client closing (subscriber context canceled)
logger.Info("stop subscribe from inner context canceled",
zap.String("subscriber-id", s.id),
zap.String("topic", topicName),
zap.String("publisher-endpoint", pubEndpoint))
} else if status.Code(err) == codes.Unavailable { // server closing (publisher context canceled)
logger.Info("stop subscribe from publisher closed",
zap.String("subscriber-id", s.id),
zap.String("topic", topicName),
zap.String("publisher-endpoint", pubEndpoint))
} else {
errStream <- err
}
return
}
fetchedResults := subscriptionResult.Results
logger.Debug("received",
zap.String("subscriber-id", s.id),
zap.String("topic", topicName),
zap.String("publisher-endpoint", pubEndpoint),
zap.Int("num data", len(fetchedResults)),
zap.Uint64("last seqNum", fetchedResults[len(fetchedResults)-1].SeqNum))
var results []SubscriptionResult
for _, result := range fetchedResults {
results = append(results, SubscriptionResult{
FragmentId: uint(result.FragmentId),
SeqNum: result.SeqNum,
Data: result.Data,
})
s.lastSubscribedOffset.Store(storage.NewFragmentKey(topicName, uint(result.FragmentId)), result.Offset)
}
select {
case <-ctx.Done():
logger.Info("stop subscribe from ctx.Done()",
zap.String("subscriber-id", s.id),
zap.String("topic", topicName),
zap.String("publisher-endpoint", pubEndpoint))
return
default:
outStream <- results
}
runtime.Gosched()
}
}
}(endpoint)
}
// wait for all subscription to be finished
subscriptionWg.Add(1)
go func() {
defer subscriptionWg.Done()
defer close(errStream)
defer close(outStream)
wg.Wait()
logger.Info("all subscription streams closed", zap.String("subscriber-id", s.id), zap.String("topic", topicName), zap.Uints("fragmentIds", subscriptionFragments))
}()
return outStream, errStream, nil
}
func (s *Subscriber) Wait() {
s.wg.Wait()
}