forked from dropbox/marshal
-
Notifications
You must be signed in to change notification settings - Fork 0
/
marshal.go
420 lines (362 loc) · 14.2 KB
/
marshal.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
/*
* portal - marshal
*
* a library that implements an algorithm for doing consumer coordination within Kafka, rather
* than using Zookeeper or another external system.
*
*/
package marshal
import (
"fmt"
"sync"
"sync/atomic"
"time"
"github.com/dropbox/kafka"
"github.com/dropbox/kafka/proto"
"github.com/pborman/uuid"
)
const (
// MarshalTopic is the main topic used for coordination. This must be constant across all
// consumers that you want to coordinate.
MarshalTopic = "__marshal"
// HeartbeatInterval is the main timing used to determine how "chatty" the system is and how
// fast it responds to failures of consumers. THIS VALUE MUST BE THE SAME BETWEEN ALL CONSUMERS
// as it is critical to coordination.
HeartbeatInterval = 60 // Measured in seconds.
)
// Marshaler is the coordinator type. It is designed to be used once per (client,
// group) and is thread safe. Creating one of these will create connections to your
// Kafka cluster and begin actively monitoring the coordination topic.
type Marshaler struct {
// These members are not protected by the lock and can be read at any
// time as they're write-once or only ever atomically updated. They must
// never be overwritten once a Marshaler is created.
quit *int32
cluster *KafkaCluster
ownsCluster bool
instanceID string
clientID string
groupID string
offsets kafka.OffsetCoordinator
// Lock protects the following members; you must have this lock in order to
// read from or write to these.
lock sync.RWMutex
consumers []*Consumer
}
// NewMarshaler connects to a cluster (given broker addresses) and prepares to handle marshalling
// requests. Given the way this system works, the marshaler has to process all messages in the
// topic before it's safely able to begin operating. This might take a while. NOTE: If you are
// creating multiple marshalers in your program, you should instead call Dial and then use
// the NewMarshaler method on that object.
func NewMarshaler(clientID, groupID string, brokers []string) (*Marshaler, error) {
cluster, err := Dial("automatic", brokers)
if err != nil {
return nil, err
}
m, err := cluster.NewMarshaler(clientID, groupID)
if err != nil {
m.ownsCluster = true
}
return m, err
}
// newInstanceID creates a new random instance ID for use inside Marshal messages. This
// is generated new every time we restart.
func newInstanceID() string {
// A UUID4 starts with 8 random characters, so let's use that as our instance ID.
// This should be a good tradeoff between randomness and brevity.
return uuid.New()[0:8]
}
// addNewConsumer is called when a new Consumer is created. This allows Marshal to keep
// track of the consumers that exist so we can operate on them later if needed.
func (m *Marshaler) addNewConsumer(c *Consumer) {
m.lock.Lock()
defer m.lock.Unlock()
m.consumers = append(m.consumers, c)
}
// removeConsumer is called when a Consumer is terminating and should be removed from our list.
func (m *Marshaler) removeConsumer(c *Consumer) {
m.lock.Lock()
defer m.lock.Unlock()
for i, cn := range m.consumers {
if cn == c {
m.consumers = append(m.consumers[:i], m.consumers[i+1:]...)
break
}
}
}
// getClaimedPartitionState returns a topicState iff it is claimed by the current Marshaler.
// Else, an error is returned. This is on the Marshaler becomes it's a helper to only return
// a claim that is presently valid and owned by us.
func (m *Marshaler) getClaimedPartitionState(topicName string, partID int) (
*topicState, error) {
// Get partition state of whatever happens to be here
topic := m.cluster.getPartitionState(m.groupID, topicName, partID)
topic.lock.RLock()
defer topic.lock.RUnlock()
if !topic.partitions[partID].isClaimed(m.cluster.ts) {
return nil, fmt.Errorf("Partition %s:%d is not claimed!", topicName, partID)
}
// And if it's not claimed by us...
if topic.partitions[partID].GroupID != m.groupID ||
topic.partitions[partID].ClientID != m.clientID {
return nil, fmt.Errorf("Partition %s:%d is not claimed by us!", topicName, partID)
}
return topic, nil
}
// Topics returns the list of known topics.
func (m *Marshaler) Topics() []string {
return m.cluster.getTopics()
}
// Partitions returns the count of how many partitions are in a given topic. Returns 0 if a
// topic is unknown.
func (m *Marshaler) Partitions(topicName string) int {
return m.cluster.getTopicPartitions(topicName)
}
// terminateAndCleanup terminates the marshal, with the option of removing
// the marshaler's reference from its associated cluster.
func (m *Marshaler) terminateAndCleanup(remove bool) {
if !atomic.CompareAndSwapInt32(m.quit, 0, 1) {
return
}
m.lock.Lock()
defer m.lock.Unlock()
// Now terminate all of the consumers. In this codepath we do a no-release termination
// because that is usually correct in production. If someone actually wants to release
// they need to terminate the consumers manually.
for _, cn := range m.consumers {
cn.terminateAndCleanup(false, false)
}
m.consumers = nil
// If we own the cluster, terminate it.
if m.ownsCluster {
m.cluster.Terminate()
}
// Remove this marshal from its cluster. Doing so is recommended
// if the cluster doesn't remove the terminated marshal itself (by setting its
// list of marshals to nil or filtering them).
if remove {
m.cluster.removeMarshal(m)
}
}
// Terminate is called when we're done with the marshaler and want to shut down.
func (m *Marshaler) Terminate() {
m.terminateAndCleanup(true)
}
// Terminated returns whether or not we have been terminated.
func (m *Marshaler) Terminated() bool {
return atomic.LoadInt32(m.quit) == 1
}
// IsClaimed returns the current status on whether or not a partition is claimed by any other
// consumer in our group (including ourselves). A topic/partition that does not exist is
// considered to be unclaimed.
func (m *Marshaler) IsClaimed(topicName string, partID int) bool {
// The contract of this method is that if it returns something and the heartbeat is
// non-zero, the partition is claimed.
claim := m.GetPartitionClaim(topicName, partID)
return claim.LastHeartbeat > 0
}
// GetPartitionClaim returns a PartitionClaim structure for a given partition. The structure
// describes the consumer that is currently claiming this partition. This is a copy of the
// claim structure, so changing it cannot change the world state.
func (m *Marshaler) GetPartitionClaim(topicName string, partID int) PartitionClaim {
topic := m.cluster.getPartitionState(m.groupID, topicName, partID)
topic.lock.RLock()
defer topic.lock.RUnlock()
if topic.partitions[partID].isClaimed(m.cluster.ts) {
return topic.partitions[partID] // copy.
}
return PartitionClaim{}
}
// GetLastPartitionClaim returns a PartitionClaim structure for a given partition. The structure
// describes the consumer that is currently or most recently claiming this partition. This is a
// copy of the claim structure, so changing it cannot change the world state.
func (m *Marshaler) GetLastPartitionClaim(topicName string, partID int) PartitionClaim {
topic := m.cluster.getPartitionState(m.groupID, topicName, partID)
topic.lock.RLock()
defer topic.lock.RUnlock()
return topic.partitions[partID] // copy.
}
// GetPartitionOffsets returns the current state of a topic/partition. This has to hit Kafka
// thrice to ask about a partition, but it returns the full state of information that can be
// used to calculate consumer lag.
func (m *Marshaler) GetPartitionOffsets(topicName string, partID int) (PartitionOffsets, error) {
var err error
o := PartitionOffsets{}
o.Earliest, err = m.cluster.broker.OffsetEarliest(topicName, int32(partID))
if err != nil {
return PartitionOffsets{}, err
}
o.Latest, err = m.cluster.broker.OffsetLatest(topicName, int32(partID))
if err != nil {
return PartitionOffsets{}, err
}
// Get committed offsets for our particular group using our offset coordinator.
o.Committed, _, err = m.offsets.Offset(topicName, int32(partID))
if err != nil {
// This error happens when Kafka does not know about the partition i.e. no
// offset has been committed here. In that case we ignore it.
if err != proto.ErrUnknownTopicOrPartition {
return PartitionOffsets{}, fmt.Errorf("offset fetch fail: %s", err)
}
}
// Use the last claim we know about, whatever it is
claim := m.GetLastPartitionClaim(topicName, partID)
o.Current = claim.LastOffset
return o, nil
}
// msgBase constructs a base message object for a message.
func (m *Marshaler) msgBase(topicName string, partID int) *msgBase {
return &msgBase{
Time: int(time.Now().Unix()),
InstanceID: m.instanceID,
ClientID: m.clientID,
GroupID: m.groupID,
Topic: topicName,
PartID: partID,
}
}
// ClaimPartition is how you can actually claim a partition. If you call this, Marshal will
// attempt to claim the partition on your behalf. This is the low level function, you probably
// want to use a MarshaledConsumer. Returns a bool on whether or not the claim succeeded and
// whether you can continue.
func (m *Marshaler) ClaimPartition(topicName string, partID int) bool {
topic := m.cluster.getPartitionState(m.groupID, topicName, partID)
// Unlock is later, since this function might take a while
// TODO: Move this logic to a func and defer the lock (for sanity sake)
topic.lock.Lock()
// If the topic is already claimed, we can short circuit the decision process
if topic.partitions[partID].isClaimed(m.cluster.ts) {
defer topic.lock.Unlock()
if topic.partitions[partID].GroupID == m.groupID &&
topic.partitions[partID].ClientID == m.clientID {
return true
}
log.Warningf("Attempt to claim already claimed partition.")
return false
}
// Make a channel for results, append it to the list so we hear about claims
out := make(chan struct{}, 1)
topic.partitions[partID].pendingClaims = append(
topic.partitions[partID].pendingClaims, out)
topic.lock.Unlock()
// Produce message to kafka
cl := &msgClaimingPartition{
msgBase: *m.msgBase(topicName, partID),
}
_, err := m.cluster.producer.Produce(MarshalTopic, int32(topic.claimPartition),
&proto.Message{Value: []byte(cl.Encode())})
if err != nil {
// If we failed to produce, this is probably serious so we should undo the work
// we did and then return failure
log.Errorf("Failed to produce to Kafka: %s", err)
return false
}
// Wait for channel to close, which is the signal that the rationalizer has
// updated the status.
<-out
// Now we have to check if we own the partition. If this returns anything, the partition
// is ours. nil = not.
topic, err = m.getClaimedPartitionState(topicName, partID)
if topic == nil || err != nil {
return false
}
return true
}
// Heartbeat will send an update for other people to know that we're still alive and
// still owning this partition. Returns an error if anything has gone wrong (at which
// point we can no longer assert we have the lock).
func (m *Marshaler) Heartbeat(topicName string, partID int, lastOffset int64) error {
topic, err := m.getClaimedPartitionState(topicName, partID)
if err != nil {
return err
}
// All good, let's heartbeat
cl := &msgHeartbeat{
msgBase: *m.msgBase(topicName, partID),
LastOffset: lastOffset,
}
_, err = m.cluster.producer.Produce(MarshalTopic, int32(topic.claimPartition),
&proto.Message{Value: []byte(cl.Encode())})
if err != nil {
return fmt.Errorf("Failed to produce heartbeat to Kafka: %s", err)
}
err = m.CommitOffsets(topicName, partID, lastOffset)
return err
}
// ReleasePartition will send an update for other people to know that we're done with
// a partition. Returns an error if anything has gone wrong (at which
// point we can no longer assert we have the lock).
func (m *Marshaler) ReleasePartition(topicName string, partID int, lastOffset int64) error {
topic, err := m.getClaimedPartitionState(topicName, partID)
if err != nil {
return err
}
// All good, let's release
cl := &msgReleasingPartition{
msgBase: *m.msgBase(topicName, partID),
LastOffset: lastOffset,
}
_, err = m.cluster.producer.Produce(MarshalTopic, int32(topic.claimPartition),
&proto.Message{Value: []byte(cl.Encode())})
if err != nil {
return fmt.Errorf("Failed to produce release to Kafka: %s", err)
}
err = m.CommitOffsets(topicName, partID, lastOffset)
return err
}
// CommitOffsets will commit the partition offsets to Kafka so it's available in the
// long-term storage of the offset coordination system. Note: this method does not ensure
// that this Marshal instance owns the topic/partition in question.
func (m *Marshaler) CommitOffsets(topicName string, partID int, lastOffset int64) error {
err := m.offsets.Commit(topicName, int32(partID), lastOffset)
if err != nil {
// Do not count this as a returned error as that will cause us to drop consumption, but
// do log it so people can see it
log.Errorf("[%s:%d] failed to commit offsets: %s", topicName, partID, err)
}
return nil
}
// ClientID returns the client ID we're using
func (m *Marshaler) ClientID() string {
return m.clientID
}
// GroupID returns the group ID we're using
func (m *Marshaler) GroupID() string {
return m.groupID
}
// PrintState will take the current state of the Marshal world and print it verbosely to the
// logging output. This is used in the rare case where we're self-terminating or on request
// from the user.
func (m *Marshaler) PrintState() {
m.lock.RLock()
defer m.lock.RUnlock()
m.cluster.lock.RLock()
defer m.cluster.lock.RUnlock()
log.Infof("Marshal state dump beginning.")
log.Infof("")
log.Infof("Group ID: %s", m.groupID)
log.Infof("Client ID: %s", m.clientID)
log.Infof("Instance ID: %s", m.instanceID)
log.Infof("")
log.Infof("Marshal topic partitions: %d", m.cluster.partitions)
log.Infof("Known Kafka topics: %d", len(m.cluster.topics))
log.Infof("Internal rsteps counter: %d", atomic.LoadInt32(m.cluster.rsteps))
log.Infof("")
log.Infof("State of the world:")
log.Infof("")
for group, topicmap := range m.cluster.groups {
log.Infof(" GROUP: %s", group)
for topic, state := range topicmap {
log.Infof(" TOPIC: %s [on %s:%d]", topic, MarshalTopic, state.claimPartition)
state.PrintState()
}
}
log.Infof("")
log.Infof("Consumer states:")
log.Infof("")
for _, consumer := range m.consumers {
consumer.PrintState()
}
log.Infof("")
log.Infof("Marshal state dump complete.")
}