forked from twitchscience/kinsumer
-
Notifications
You must be signed in to change notification settings - Fork 0
/
leader.go
336 lines (309 loc) · 9.83 KB
/
leader.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
// Copyright (c) 2016 Twitch Interactive
package kinsumer
import (
"fmt"
"sort"
"time"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/awserr"
"github.com/aws/aws-sdk-go/service/dynamodb"
"github.com/aws/aws-sdk-go/service/dynamodb/dynamodbattribute"
"github.com/aws/aws-sdk-go/service/dynamodb/dynamodbiface"
"github.com/aws/aws-sdk-go/service/kinesis"
"github.com/aws/aws-sdk-go/service/kinesis/kinesisiface"
)
const (
leaderKey = "Leader"
shardCacheKey = "ShardCache"
conditionalFail = "ConditionalCheckFailedException"
)
type shardCacheRecord struct {
Key string // must be "ShardCache"
ShardIDs []string // Slice of unfinished shard IDs
LastUpdate int64 // timestamp of last update
// Debug versions of LastUpdate
LastUpdateRFC string
}
// becomeLeader starts the leadership goroutine with a channel to stop it.
// TODO(dwe): Factor out dependencies and unit test
func (k *Kinsumer) becomeLeader() {
if k.isLeader {
return
}
k.leaderLost = make(chan bool)
k.leaderWG.Add(1)
go func() {
defer k.leaderWG.Done()
leaderActions := time.NewTicker(k.config.leaderActionFrequency)
defer func() {
leaderActions.Stop()
err := k.deregisterLeadership()
if err != nil {
k.errors <- fmt.Errorf("error deregistering leadership: %v", err)
}
}()
ok, err := k.registerLeadership()
if err != nil {
k.errors <- fmt.Errorf("error registering initial leadership: %v", err)
}
// Perform leadership actions immediately if we became leader. If we didn't
// become leader yet, wait until the first tick to try again.
if ok {
err = k.performLeaderActions()
if err != nil {
k.errors <- fmt.Errorf("error performing initial leader actions: %v", err)
}
}
for {
select {
case <-leaderActions.C:
ok, err := k.registerLeadership()
if err != nil {
k.errors <- fmt.Errorf("error registering leadership: %v", err)
}
if !ok {
continue
}
err = k.performLeaderActions()
if err != nil {
k.errors <- fmt.Errorf("error performing repeated leader actions: %v", err)
}
case <-k.leaderLost:
return
}
}
}()
k.isLeader = true
}
// unbecomeLeader stops the leadership goroutine.
func (k *Kinsumer) unbecomeLeader() {
if !k.isLeader {
return
}
if k.leaderLost == nil {
k.config.logger.Log("Lost leadership but k.leaderLost was nil")
} else {
close(k.leaderLost)
k.leaderWG.Wait()
k.leaderLost = nil
}
k.isLeader = false
}
// performLeaderActions updates the shard ID cache and reaps old clients
// TODO(dwe): Factor out dependencies and unit test
func (k *Kinsumer) performLeaderActions() error {
shardCache, err := loadShardCacheFromDynamo(k.dynamodb, k.metadataTableName)
if err != nil {
return fmt.Errorf("error loading shard cache from dynamo: %v", err)
}
cachedShardIDs := shardCache.ShardIDs
now := time.Now().UnixNano()
if now-shardCache.LastUpdate < k.config.leaderActionFrequency.Nanoseconds() {
return nil
}
curShardIDs, err := loadShardIDsFromKinesis(k.kinesis, k.streamName)
if err != nil {
return fmt.Errorf("error loading shard IDs from kinesis: %v", err)
}
checkpoints, err := loadCheckpoints(k.dynamodb, k.checkpointTableName)
if err != nil {
return fmt.Errorf("error loading shard IDs from dynamo: %v", err)
}
updatedShardIDs, changed := diffShardIDs(curShardIDs, cachedShardIDs, checkpoints)
if changed {
err = k.setCachedShardIDs(updatedShardIDs)
if err != nil {
return fmt.Errorf("error caching shard IDs to dynamo: %v", err)
}
}
err = reapClients(k.dynamodb, k.clientsTableName)
if err != nil {
return fmt.Errorf("error reaping old clients: %v", err)
}
return nil
}
// setCachedShardIDs updates the shard ID cache in dynamo.
func (k *Kinsumer) setCachedShardIDs(shardIDs []string) error {
if len(shardIDs) == 0 {
return nil
}
now := time.Now()
item, err := dynamodbattribute.MarshalMap(&shardCacheRecord{
Key: shardCacheKey,
ShardIDs: shardIDs,
LastUpdate: now.UnixNano(),
LastUpdateRFC: now.UTC().Format(time.RFC1123Z),
})
if err != nil {
return fmt.Errorf("error marshalling map: %v", err)
}
_, err = k.dynamodb.PutItem(&dynamodb.PutItemInput{
TableName: aws.String(k.metadataTableName),
Item: item,
})
if err != nil {
return fmt.Errorf("error updating shard cache: %v", err)
}
return nil
}
// diffShardIDs takes the current shard IDs and cached shards and returns the new sorted cache, ignoring
// finished shards correctly.
func diffShardIDs(curShardIDs, cachedShardIDs []string, checkpoints map[string]*checkpointRecord) (updatedShardIDs []string, changed bool) {
// Look for differences, ignoring Finished shards.
cur := make(map[string]bool)
for _, s := range curShardIDs {
cur[s] = true
}
for _, s := range cachedShardIDs {
if cur[s] {
delete(cur, s)
// Drop the shard if it's been finished.
if c, ok := checkpoints[s]; ok && c.Finished != nil {
changed = true
} else {
updatedShardIDs = append(updatedShardIDs, s)
}
} else {
// If a shard is no longer returned by DescribeStream, drop it.
changed = true
}
}
for s := range cur {
// If the shard is returned by DescribeStream and not already Finished, add it.
if c, ok := checkpoints[s]; !ok || c.Finished == nil {
updatedShardIDs = append(updatedShardIDs, s)
changed = true
}
}
sort.Strings(updatedShardIDs)
return
}
// deregisterLeadership marks us as no longer the leader in dynamo.
func (k *Kinsumer) deregisterLeadership() error {
now := time.Now()
attrVals, err := dynamodbattribute.MarshalMap(map[string]interface{}{
":ID": aws.String(k.clientID),
":lastUpdate": aws.Int64(now.UnixNano()),
":lastUpdateRFC": aws.String(now.UTC().Format(time.RFC1123Z)),
})
if err != nil {
return fmt.Errorf("error marshaling deregisterLeadership ExpressionAttributeValues: %v", err)
}
_, err = k.dynamodb.UpdateItem(&dynamodb.UpdateItemInput{
TableName: aws.String(k.metadataTableName),
Key: map[string]*dynamodb.AttributeValue{
"Key": {S: aws.String(leaderKey)},
},
ConditionExpression: aws.String("ID = :ID"),
UpdateExpression: aws.String("REMOVE ID SET LastUpdate = :lastUpdate, LastUpdateRFC = :lastUpdateRFC"),
ExpressionAttributeValues: attrVals,
})
if err != nil {
// It's ok if we never actually became leader.
if awsErr, ok := err.(awserr.Error); ok && awsErr.Code() == conditionalFail {
return nil
}
}
return err
}
// registerLeadership marks us as the leader or just refreshes LastUpdate in dynamo, returning false if
// another node is the leader.
func (k *Kinsumer) registerLeadership() (bool, error) {
now := time.Now()
cutoff := now.Add(-k.maxAgeForLeaderRecord).UnixNano()
attrVals, err := dynamodbattribute.MarshalMap(map[string]interface{}{
":ID": aws.String(k.clientID),
":cutoff": aws.Int64(cutoff),
})
if err != nil {
return false, fmt.Errorf("error marshaling registerLeadership ExpressionAttributeValues: %v", err)
}
item, err := dynamodbattribute.MarshalMap(map[string]interface{}{
"Key": aws.String(leaderKey),
"ID": aws.String(k.clientID),
"Name": aws.String(k.clientName),
"LastUpdate": aws.Int64(now.UnixNano()),
"LastUpdateRFC": aws.String(now.UTC().Format(time.RFC1123Z)),
})
if err != nil {
return false, fmt.Errorf("error marshaling registerLeadership Item: %v", err)
}
_, err = k.dynamodb.PutItem(&dynamodb.PutItemInput{
TableName: aws.String(k.metadataTableName),
Item: item,
ConditionExpression: aws.String("ID = :ID OR attribute_not_exists(ID) OR LastUpdate <= :cutoff"),
ExpressionAttributeValues: attrVals,
})
if err != nil {
if awsErr, ok := err.(awserr.Error); ok && awsErr.Code() == conditionalFail {
return false, nil
}
return false, err
}
return true, nil
}
// loadShardIDsFromKinesis returns a sorted slice of shardIDs from kinesis.
// This function uses kinesis.DescribeStream, which has a very low throttling limit of 10/s per account.
// To avoid hitting that limit, unless you need an as-recent-as-possible list,
// you should use the cache, returned by loadShardIDsFromDynamo below.
//TODO: Write unit test - needs kinesis mocking
func loadShardIDsFromKinesis(kin kinesisiface.KinesisAPI, streamName string) ([]string, error) {
var innerError error
res, err := kin.ListShards(&kinesis.ListShardsInput{
StreamName: aws.String(streamName),
})
if err != nil {
if e, ok := err.(awserr.Error); ok {
switch e.Code() {
case "ResourceInUseException":
innerError = ErrStreamBusy
case "ResourceNotFoundException":
innerError = ErrNoSuchStream
}
}
}
if innerError != nil {
return nil, innerError
}
if err != nil {
return nil, err
}
shardIDs := make([]string, len(res.Shards))
for i, s := range res.Shards {
shardIDs[i] = aws.StringValue(s.ShardId)
}
sort.Strings(shardIDs)
return shardIDs, nil
}
// loadShardIDsFromDynamo returns the sorted slice of shardIDs from the metadata table in dynamo.
func loadShardIDsFromDynamo(db dynamodbiface.DynamoDBAPI, tableName string) ([]string, error) {
record, err := loadShardCacheFromDynamo(db, tableName)
if err != nil {
return nil, err
}
if record == nil {
return nil, nil
}
return record.ShardIDs, nil
}
// loadShardCacheFromDynamo returns the ShardCache record from the metadata table in dynamo.
func loadShardCacheFromDynamo(db dynamodbiface.DynamoDBAPI, tableName string) (*shardCacheRecord, error) {
resp, err := db.GetItem(&dynamodb.GetItemInput{
TableName: aws.String(tableName),
ConsistentRead: aws.Bool(true),
Key: map[string]*dynamodb.AttributeValue{
"Key": {S: aws.String(shardCacheKey)},
},
})
if err != nil {
if awsErr, ok := err.(awserr.Error); ok && awsErr.Code() == "ResourceNotFoundException" {
return nil, nil
}
return nil, err
}
var record shardCacheRecord
if err = dynamodbattribute.UnmarshalMap(resp.Item, &record); err != nil {
return nil, err
}
return &record, nil
}