forked from asonawalla/gazette
/
runner.go
250 lines (209 loc) · 6.78 KB
/
runner.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
package consumer
import (
"path"
"sort"
etcd "github.com/coreos/etcd/client"
log "github.com/sirupsen/logrus"
"golang.org/x/net/context"
"github.com/LiveRamp/gazette/pkg/consensus"
"github.com/LiveRamp/gazette/pkg/journal"
"github.com/LiveRamp/gazette/pkg/topic"
)
const (
// Peer is actively serving the Shard.
Primary = "primary"
// Peer is ready to immediately transition to Shard primary.
Ready = "ready"
// Peer is still rebuilding from the recovery log.
Recovering = "recovering"
// Peer is responsible for a consumer Shard it doesn't know about.
// This typically happens when topics are removed from a consumer,
// but remain in (and should be removed from) the consumer's Etcd directory.
UnknownShard = "unknown-shard"
)
type Runner struct {
Consumer Consumer
// An identifier for this particular runner. Eg, the hostname.
LocalRouteKey string
// Base local directory into which shards should be staged.
LocalDir string
// Base path in Etcd via which the consumer should coordinate.
ConsumerRoot string
// Base journal path for recovery logs of this consumer.
RecoveryLogRoot string
// Required number of replicas of the consumer.
ReplicaCount int
Etcd etcd.Client
Gazette journal.Client
// Optional hooks for notification of Shard lifecycle. These are largely
// intended to facilitate testing cases.
ShardPostInitHook func(Shard)
ShardPostConsumeHook func(topic.Envelope, Shard)
ShardPostCommitHook func(Shard)
ShardPostStopHook func(Shard)
partitions map[journal.Name]*topic.Description // Previously enumerated topic partitions.
shardNames []string // Allocator FixedItems support.
allShards map[ShardID]topic.Partition // All shards and their Partition, by name.
liveShards map[ShardID]*shard // Live shards, by name.
zombieShards map[*shard]struct{} // Cancelled shards which are shutting down.
inspectCh chan func(*etcd.Node)
}
func (r *Runner) CurrentConsumerState(context.Context, *Empty) (*ConsumerState, error) {
var out = &ConsumerState{
Root: r.ConsumerRoot,
LocalRouteKey: r.LocalRouteKey,
ReplicaCount: int32(r.ReplicaCount),
}
var doneCh = make(chan struct{})
r.inspectCh <- func(tree *etcd.Node) {
for _, n := range consensus.Child(tree, consensus.MemberPrefix).Nodes {
// Member Nodes are already sorted on node Key.
out.Endpoints = append(out.Endpoints, path.Base(n.Key))
}
consensus.WalkItems(tree, r.FixedItems(), func(name string, route consensus.Route) {
var shardID = ShardID(name)
var partition, ok = r.allShards[shardID]
if !ok {
return
}
var shard = ConsumerState_Shard{
Id: shardID,
Topic: partition.Topic.Name,
Partition: partition.Journal,
}
for _, e := range route.Entries {
var replica = ConsumerState_Replica{
Endpoint: path.Base(e.Key),
}
switch e.Value {
case Primary:
replica.Status = ConsumerState_Replica_PRIMARY
case Ready:
replica.Status = ConsumerState_Replica_READY
case Recovering:
replica.Status = ConsumerState_Replica_RECOVERING
default:
replica.Status = ConsumerState_Replica_INVALID
}
shard.Replicas = append(shard.Replicas, replica)
}
// WalkItems enumerates in sorted |name| order.
out.Shards = append(out.Shards, shard)
})
close(doneCh)
}
<-doneCh
return out, nil
}
func (r *Runner) updateShards() {
var added bool
for _, t := range r.Consumer.Topics() {
for _, j := range t.Partitions() {
if _, ok := r.partitions[j]; !ok {
r.partitions[j] = t
added = true
}
}
}
if !added {
return
}
r.allShards = EnumerateShards(r.Consumer)
var names []string
for id := range r.allShards {
names = append(names, id.String())
}
sort.Strings(names)
r.shardNames = names
}
func (r *Runner) Run() error {
if r.ConsumerRoot == "" {
log.Fatal("ConsumerRoot cannot be empty")
}
r.partitions = make(map[journal.Name]*topic.Description)
r.allShards = make(map[ShardID]topic.Partition)
r.liveShards = make(map[ShardID]*shard)
r.zombieShards = make(map[*shard]struct{})
r.inspectCh = make(chan func(*etcd.Node))
var err = consensus.CreateAndAllocateWithSignalHandling(r)
// Allocate should exit only after all shards have been cancelled.
if err == nil && len(r.liveShards) != 0 {
log.WithField("shards", r.liveShards).Panic("live shards on Allocate exit")
}
// Wait for all shards to complete teardown before returning. This gives
// shards a chance to finish background IO and close cleanly.
for name, shard := range r.liveShards {
shard.transitionCancel()
shard.blockUntilHalted()
delete(r.liveShards, name)
}
for s := range r.zombieShards {
s.blockUntilHalted()
delete(r.zombieShards, s)
}
return err
}
// consumer.Allocator implementation.
func (r *Runner) FixedItems() []string {
r.updateShards()
return r.shardNames
}
func (r *Runner) InstanceKey() string { return r.LocalRouteKey }
func (r *Runner) KeysAPI() etcd.KeysAPI { return etcd.NewKeysAPI(r.Etcd) }
func (r *Runner) PathRoot() string { return r.ConsumerRoot }
func (r *Runner) Replicas() int { return r.ReplicaCount }
func (r *Runner) ItemState(name string) string {
if shard, ok := r.liveShards[ShardID(name)]; !ok {
return UnknownShard
} else if shard.master != nil && shard.master.didFinishInit() {
return Primary
} else if shard.replica.player.IsTailing() {
return Ready
} else {
return Recovering
}
}
func (r *Runner) ItemIsReadyForPromotion(item, state string) bool {
return state == Ready
}
func (r *Runner) ItemRoute(name string, rt consensus.Route, index int, tree *etcd.Node) {
var id = ShardID(name)
var current, exists = r.liveShards[id]
// |index| captures the allocator's role in processing |current|.
var isMaster, isReplica = (index == 0), (index > 0 && index <= r.ReplicaCount)
if !exists && (isMaster || isReplica) {
var partition, ok = r.allShards[id]
if !ok {
log.WithField("shard", id).Warn("unexpected consumer shard name")
return
}
// Look for a matching zombie shard (ie, still in tear-down). This happens
// if, for example, a prior shard master gives up control and then
// immediately becomes a shard replica. Other races are possible.
var zombie *shard
for s := range r.zombieShards {
if s.id == id {
delete(r.zombieShards, s)
zombie = s
}
}
current = newShard(id, partition, r, zombie)
r.liveShards[id] = current
}
if isMaster {
current.transitionMaster(r, tree)
} else if isReplica {
current.transitionReplica(r, tree)
} else if exists {
current.transitionCancel()
delete(r.liveShards, id)
r.zombieShards[current] = struct{}{}
}
// Non-blocking reap of previously-cancelled shards.
for s := range r.zombieShards {
if s.hasHalted() {
delete(r.zombieShards, s)
}
}
}
func (r *Runner) InspectChan() chan func(*etcd.Node) { return r.inspectCh }