-
Notifications
You must be signed in to change notification settings - Fork 661
/
processor.go
350 lines (287 loc) · 9.64 KB
/
processor.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
/**
* Copyright 2020 Confluent Inc.
*
* 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 main
// This is the processor, consuming the input topic of ingress car messages,
// deciding what traffic light to turn green for each handled intersection,
// and emitting the traffic light states to the output topic, all using the
// transactional API.
import (
"encoding/json"
"fmt"
"sync"
"time"
"github.com/confluentinc/confluent-kafka-go/v2/kafka"
)
// The processor's consumer group id.
var processorGroupID = "go-transactions-example-processor"
// intersectionState
type intersectionState struct {
name string // Name of intersection
partition int32 // Input partition
lightState map[string]string // Current light states, indexed by road
carsWaiting map[string]int // Numbers of cars waiting, indexed by road
carCnt int // Total number of cars waiting
currGreen string // Currently green road
lastChange time.Time // Time of last light change
}
// intersectionStates maintains state per intersection
var intersectionStates map[string]*intersectionState
// producers map assigned consumer input partitions to Producer instances.
var producers map[int32]*kafka.Producer
// consumer is the processor consumer
var processorConsumer *kafka.Consumer
// lightStateMsg is the state representation of a traffic light and ingress road
// as sent on the output topic enveloped in intersectionStateMsg.
type lightStateMsg struct {
Name string // Name of intersection
Road string // Name of ingress road, this in combination with Name identifies a single light
State string // red, green
CarsWaiting int // Number of cars waiting at this ingress
}
// intersectionStateMsg is the state representation of an intersection
// with its traffic lights as sent on the output topic as JSON.
type intersectionStateMsg struct {
Name string // Name of intersection
Lights map[string]lightStateMsg // Per traffic light state
}
func (m lightStateMsg) String() string {
return fmt.Sprintf("lightState{%s: %s is %s, %d cars waiting}",
m.Name, m.Road, m.State, m.CarsWaiting)
}
func getIntersectionState(name string, partition int32) *intersectionState {
istate, found := intersectionStates[name]
if found {
return istate
}
istate = &intersectionState{
name: name,
partition: partition,
}
istate.lightState = make(map[string]string)
istate.carsWaiting = make(map[string]int)
for _, light := range Roads {
istate.lightState[light] = "red"
istate.carsWaiting[light] = 0
}
intersectionStates[name] = istate
return istate
}
// getNextRoad returns the next road in a cyclical fashion.
func getNextRoad(currRoad string) string {
for i, road := range Roads {
if currRoad == road {
return Roads[(i+1)%len(Roads)]
}
}
return Roads[0]
}
// processIngressCarMessage processes an input message of a car coming
// into an intersection.
func processIngressCarMessage(msg *kafka.Message) {
if msg.Key == nil || msg.Value == nil {
// Invalid message, ignore
return
}
intersection := string(msg.Key)
road := string(msg.Value)
istate := getIntersectionState(intersection, msg.TopicPartition.Partition)
_, found := istate.carsWaiting[road]
if !found {
addLog(fmt.Sprintf("Processor: %v: unknown road \"%s\" for intersection \"%s\": ignoring",
msg.TopicPartition, road, istate.name))
return
}
if istate.currGreen != road {
istate.carsWaiting[road]++
istate.carCnt++
}
// Keep track of which input partition this istate is mapped to
// so we know which transactional producer to use.
if istate.partition == kafka.PartitionAny {
istate.partition = msg.TopicPartition.Partition
}
}
// electNewGreenLight elects a new green light based and updates
// the intersection state.
// Returns true if the light changed, else false.
func electNewGreenLight(istate *intersectionState) bool {
if istate.carCnt == 0 {
// No cars waiting at any roads
return false
}
// Created a weighted map of roads, where the weight
// is based on the fraction of cars for each ingress road,
// as well as the fair next road.
nextRoad := getNextRoad(istate.currGreen)
weightedRoads := make(map[string]float64)
weightedRoads[nextRoad] = 1.0
for road, cnt := range istate.carsWaiting {
weightedRoads[road] = 1.0 + (float64(cnt) / float64(istate.carCnt))
}
// Sorting a map by value is not straight forward in Go,
// so let's do iteratively instead.
bestWeight := 0.0
bestRoad := ""
for road, weight := range weightedRoads {
if weight > bestWeight {
bestWeight = weight
bestRoad = road
}
}
_, found := weightedRoads[istate.currGreen]
if found && istate.carsWaiting[istate.currGreen] == istate.carCnt {
// All cars are already on the already green road, do nothing
return false
}
istate.lastChange = time.Now()
istate.currGreen = bestRoad
if istate.currGreen != "" {
// Let all cars on the green road pass thru
istate.carCnt -= istate.carsWaiting[istate.currGreen]
istate.carsWaiting[istate.currGreen] = 0
}
return true
}
// Run state machine for a single intersection to update light colors.
// Returns true if output messages were produced, else false.
func intersectionStateMachine(istate *intersectionState) bool {
changed := false
// Elect new green light
if time.Since(istate.lastChange) > 4*time.Second {
changed = electNewGreenLight(istate)
}
// Get the producer for this istate's input partition
producer := producers[istate.partition]
if producer == nil {
fatal(fmt.Sprintf("BUG: No producer for intersection %s partition %v", istate.name, istate.partition))
}
// Produce message with current intersection light states.
isectMsg := intersectionStateMsg{
Name: istate.name,
Lights: make(map[string]lightStateMsg)}
for road := range istate.lightState {
if road == istate.currGreen {
istate.lightState[road] = "green"
} else {
istate.lightState[road] = "red"
}
isectMsg.Lights[road] = lightStateMsg{
Name: istate.name,
Road: road,
State: istate.lightState[road],
CarsWaiting: istate.carsWaiting[road]}
}
value, err := json.Marshal(isectMsg)
if err != nil {
fatal(err)
}
err = producer.Produce(
&kafka.Message{
TopicPartition: kafka.TopicPartition{
Topic: &outputTopic,
Partition: kafka.PartitionAny,
},
Key: []byte(istate.name),
Value: value,
}, nil)
if err != nil {
fatal(fmt.Sprintf("Failed to produce message: %v", err))
}
return changed
}
func trafficLightProcessor(wg *sync.WaitGroup, termChan chan bool) {
defer wg.Done()
doTerm := false
ticker := time.NewTicker(500 * time.Millisecond)
// If punctuate is true the current intersection state is written to
// the output topic every ticker interval regardless if there was an
// intersection light state change.
// Use this to get a more responsive visualization.
punctuate := true
intersectionStates = make(map[string]*intersectionState)
// The per-partition producers are set up in groupRebalance
producers = make(map[int32]*kafka.Producer)
consumerConfig := &kafka.ConfigMap{
"client.id": "processor",
"bootstrap.servers": bootstrapServers,
"group.id": processorGroupID,
"auto.offset.reset": "earliest",
// Consumer used for input to a transactional processor
// must have auto-commits disabled since offsets
// are committed with the transaction using
// SendOffsetsToTransaction.
"enable.auto.commit": false,
"go.logs.channel.enable": true,
"go.logs.channel": logsChan,
}
var err error
processorConsumer, err = kafka.NewConsumer(consumerConfig)
if err != nil {
fatal(err)
}
err = processorConsumer.Subscribe(inputTopic, groupRebalance)
if err != nil {
fatal(err)
}
addLog(fmt.Sprintf("Processor: waiting for messages on topic %s",
inputTopic))
for !doTerm {
select {
case <-ticker.C:
// Run intersection state machine(s) periodically
partitionsToCommit := make(map[int32]bool)
for _, istate := range intersectionStates {
if intersectionStateMachine(istate) || punctuate {
// The state machine wants its transaction committed.
// The transaction is shared among all intersectionStates
// that use the same input partition since the input
// offset that is committed along with the transaction
// applies to all intersectionStates mapped to
// that partition.
partitionsToCommit[istate.partition] = true
}
}
// Commit transactions
for partition := range partitionsToCommit {
commitTransactionForInputPartition(partition)
}
case <-termChan:
doTerm = true
default:
// Poll consumer for new messages or rebalance events.
ev := processorConsumer.Poll(100)
if ev == nil {
continue
}
switch e := ev.(type) {
case *kafka.Message:
// Process ingress car event message
processIngressCarMessage(e)
case kafka.Error:
// Errors are generally just informational.
addLog(fmt.Sprintf("Consumer error: %sn", ev))
default:
addLog(fmt.Sprintf("Consumer event: %s: ignored", ev))
}
}
}
addLog(fmt.Sprintf("Processor: shutting down"))
processorConsumer.Close()
for _, producer := range producers {
producer.AbortTransaction(nil)
producer.Close()
}
}