This repository has been archived by the owner on Sep 19, 2023. It is now read-only.
/
mirbft.go
590 lines (511 loc) · 15.8 KB
/
mirbft.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
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
/*
Copyright IBM Corp. All Rights Reserved.
SPDX-License-Identifier: Apache-2.0
*/
// Package mirbft is a consensus library, implementing the Mir BFT consensus protocol.
//
// This library can be used by applications which desire distributed, byzantine fault
// tolerant consensus on message order. Unlike many traditional consensus algorithms,
// Mir BFT is a multi-leader protocol, allowing throughput to scale with the number of
// nodes (even over WANs), where the performance of traditional consenus algorithms
// begin to degrade.
package mirbft
import (
"context"
"fmt"
"sync"
"time"
"github.com/IBM/mirbft/pkg/pb/msgs"
"github.com/IBM/mirbft/pkg/pb/state"
"github.com/IBM/mirbft/pkg/processor"
"github.com/IBM/mirbft/pkg/statemachine"
"github.com/IBM/mirbft/pkg/status"
"github.com/pkg/errors"
)
var ErrStopped = fmt.Errorf("stopped at caller request")
type replicas struct {
mutex sync.Mutex
eventC chan *statemachine.EventList
replicas processor.Replicas
}
func (r *replicas) replica(id uint64) *processor.Replica {
r.mutex.Lock()
defer r.mutex.Unlock()
return r.replicas.Replica(id)
}
// Node is the local instance of the MirBFT state machine through which the calling application
// proposes new messages, receives delegated actions, and returns action results.
// The methods exposed on Node are all thread safe, though typically, a single loop handles
// reading Actions, writing results, and writing ticks, while other go routines Propose and Step.
type Node struct {
ID uint64
Config *Config
processorConfig *ProcessorConfig
replicas *replicas
stateMachine *statemachine.StateMachine
workItems *processor.WorkItems
workErrNotifier *workErrNotifier
statusC chan chan *status.StateMachine
walActionsC chan *statemachine.ActionList
walResultsC chan *statemachine.ActionList
clientActionsC chan *statemachine.ActionList
clientResultsC chan *statemachine.EventList
hashActionsC chan *statemachine.ActionList
hashResultsC chan *statemachine.EventList
netActionsC chan *statemachine.ActionList
netResultsC chan *statemachine.EventList
appActionsC chan *statemachine.ActionList
appResultsC chan *statemachine.EventList
reqStoreEventsC chan *statemachine.EventList
reqStoreResultsC chan *statemachine.EventList
// Exported as a temporary hack
ResultEventsC chan *statemachine.EventList
ResultResultsC chan *statemachine.ActionList
Clients *processor.Clients
}
func StandardInitialNetworkState(nodeCount int, clientCount int) *msgs.NetworkState {
nodes := []uint64{}
for i := 0; i < nodeCount; i++ {
nodes = append(nodes, uint64(i))
}
numberOfBuckets := int32(nodeCount)
checkpointInterval := numberOfBuckets * 5
maxEpochLength := checkpointInterval * 10
clients := make([]*msgs.NetworkState_Client, clientCount)
for i := range clients {
clients[i] = &msgs.NetworkState_Client{
Id: uint64(i),
Width: 100,
LowWatermark: 0,
}
}
return &msgs.NetworkState{
Config: &msgs.NetworkState_Config{
Nodes: nodes,
F: int32((nodeCount - 1) / 3),
NumberOfBuckets: numberOfBuckets,
CheckpointInterval: checkpointInterval,
MaxEpochLength: uint64(maxEpochLength),
},
Clients: clients,
}
}
// NewNode creates a new node. The processor must be started either by invoking
// node.Processor.StartNewNode with the initial state or by invoking node.Processor.RestartNode.
func NewNode(
id uint64,
config *Config,
processorConfig *ProcessorConfig,
) (*Node, error) {
return &Node{
ID: id,
Config: config,
processorConfig: processorConfig,
replicas: &replicas{
eventC: make(chan *statemachine.EventList),
},
stateMachine: &statemachine.StateMachine{
Logger: logAdapter{Logger: config.Logger},
},
Clients: &processor.Clients{
RequestStore: processorConfig.RequestStore,
Hasher: processorConfig.Hasher,
},
workItems: processor.NewWorkItems(),
workErrNotifier: newWorkErrNotifier(),
statusC: make(chan chan *status.StateMachine),
walActionsC: make(chan *statemachine.ActionList),
walResultsC: make(chan *statemachine.ActionList),
clientActionsC: make(chan *statemachine.ActionList),
clientResultsC: make(chan *statemachine.EventList),
hashActionsC: make(chan *statemachine.ActionList),
hashResultsC: make(chan *statemachine.EventList),
netActionsC: make(chan *statemachine.ActionList),
netResultsC: make(chan *statemachine.EventList),
appActionsC: make(chan *statemachine.ActionList),
appResultsC: make(chan *statemachine.EventList),
reqStoreEventsC: make(chan *statemachine.EventList),
reqStoreResultsC: make(chan *statemachine.EventList),
ResultEventsC: make(chan *statemachine.EventList),
ResultResultsC: make(chan *statemachine.ActionList),
}, nil
}
// Status returns a static snapshot in time of the internal state of the state machine.
// This method necessarily exposes some of the internal architecture of the system, and
// especially while the library is in development, the data structures may change substantially.
// This method returns a nil status and an error if the context ends. If the serializer go routine
// exits for any other reason, then a best effort is made to return the last (and final) status.
// This final status may be relied upon if it is non-nil. If the serializer exited at the user's
// request (because the done channel was closed), then ErrStopped is returned.
func (n *Node) Status(ctx context.Context) (*status.StateMachine, error) {
statusC := make(chan *status.StateMachine, 1)
select {
case <-ctx.Done():
return nil, ctx.Err()
case <-n.workErrNotifier.ExitStatusC():
return n.workErrNotifier.ExitStatus()
case n.statusC <- statusC:
}
select {
case <-ctx.Done():
return nil, ctx.Err()
case <-n.workErrNotifier.ExitStatusC():
return n.workErrNotifier.ExitStatus()
case s := <-statusC:
return s, nil
}
}
func (n *Node) Step(ctx context.Context, source uint64, msg *msgs.Msg) error {
r := n.replicas.replica(source)
e, err := r.Step(msg)
if err != nil {
return err
}
select {
case n.replicas.eventC <- e:
return nil
case <-n.workErrNotifier.ExitStatusC():
return n.workErrNotifier.Err()
case <-ctx.Done():
return ctx.Err()
}
}
func (n *Node) doWALWork(exitC <-chan struct{}) error {
var actions *statemachine.ActionList
select {
case actions = <-n.walActionsC:
case <-exitC:
return ErrStopped
}
walResults, err := processor.ProcessWALActions(n.processorConfig.WAL, actions)
if err != nil {
return errors.WithMessage(err, "could not perform WAL actions")
}
if walResults.Len() == 0 {
return nil
}
select {
case n.walResultsC <- walResults:
case <-exitC:
return ErrStopped
}
return nil
}
func (n *Node) doClientWork(exitC <-chan struct{}) error {
var actions *statemachine.ActionList
select {
case actions = <-n.clientActionsC:
case <-exitC:
return ErrStopped
}
clientResults, err := n.Clients.ProcessClientActions(actions)
if err != nil {
return errors.WithMessage(err, "could not perform client actions")
}
if clientResults.Len() == 0 {
return nil
}
select {
case n.clientResultsC <- clientResults:
case <-exitC:
return ErrStopped
}
return nil
}
func (n *Node) doHashWork(exitC <-chan struct{}) error {
var actions *statemachine.ActionList
select {
case actions = <-n.hashActionsC:
case <-exitC:
return ErrStopped
}
hashResults, err := processor.ProcessHashActions(n.processorConfig.Hasher, actions)
if err != nil {
return errors.WithMessage(err, "could not perform hash actions")
}
select {
case n.hashResultsC <- hashResults:
case <-exitC:
return ErrStopped
}
return nil
}
func (n *Node) doNetWork(exitC <-chan struct{}) error {
var actions *statemachine.ActionList
select {
case actions = <-n.netActionsC:
case <-exitC:
return ErrStopped
}
netResults, err := processor.ProcessNetActions(n.ID, n.processorConfig.Link, actions)
if err != nil {
return errors.WithMessage(err, "could not perform net actions")
}
select {
case n.netResultsC <- netResults:
case <-exitC:
return ErrStopped
}
return nil
}
func (n *Node) doAppWork(exitC <-chan struct{}) error {
var actions *statemachine.ActionList
select {
case actions = <-n.appActionsC:
case <-exitC:
return ErrStopped
}
appResults, err := processor.ProcessAppActions(n.processorConfig.App, actions)
if err != nil {
return errors.WithMessage(err, "could not perform app actions")
}
select {
case n.appResultsC <- appResults:
case <-exitC:
return ErrStopped
}
return nil
}
func (n *Node) doReqStoreWork(exitC <-chan struct{}) error {
var events *statemachine.EventList
select {
case events = <-n.reqStoreEventsC:
case <-exitC:
return ErrStopped
}
reqStoreResults, err := processor.ProcessReqStoreEvents(n.processorConfig.RequestStore, events)
if err != nil {
return errors.WithMessage(err, "could not perform reqstore actions")
}
select {
case n.reqStoreResultsC <- reqStoreResults:
case <-exitC:
return ErrStopped
}
return nil
}
func (n *Node) doStateMachineWork(exitC <-chan struct{}) (err error) {
defer func() {
if err != nil {
s, err := n.stateMachine.Status()
n.workErrNotifier.SetExitStatus(s, err)
}
}()
var events *statemachine.EventList
select {
case events = <-n.ResultEventsC:
case <-exitC:
return ErrStopped
}
actions, err := processor.ProcessStateMachineEvents(n.stateMachine, n.processorConfig.Interceptor, events)
if err != nil {
return err
}
if actions.Len() == 0 {
return nil
}
select {
case n.ResultResultsC <- actions:
n.processorConfig.Interceptor.Intercept(statemachine.EventActionsReceived())
case <-exitC:
return ErrStopped
}
return nil
}
type workFunc func(exitC <-chan struct{}) error
func (n *Node) doUntilErr(work workFunc) {
for {
err := work(n.workErrNotifier.ExitC())
if err != nil {
n.workErrNotifier.Fail(err)
return
}
}
}
type ProcessorConfig struct {
Link processor.Link
Hasher processor.Hasher
App processor.App
WAL processor.WAL
RequestStore processor.RequestStore
Interceptor processor.EventInterceptor
}
func (n *Node) runtimeParms() *state.EventInitialParameters {
return &state.EventInitialParameters{
Id: n.ID,
BatchSize: n.Config.BatchSize,
HeartbeatTicks: n.Config.HeartbeatTicks,
SuspectTicks: n.Config.SuspectTicks,
NewEpochTimeoutTicks: n.Config.NewEpochTimeoutTicks,
BufferSize: n.Config.BufferSize,
}
}
func (n *Node) ProcessAsNewNode(
exitC <-chan struct{},
tickC <-chan time.Time,
initialNetworkState *msgs.NetworkState,
initialCheckpointValue []byte,
) error {
events, err := processor.IntializeWALForNewNode(n.processorConfig.WAL, n.runtimeParms(), initialNetworkState, initialCheckpointValue)
if err != nil {
n.workErrNotifier.SetExitStatus(nil, errors.Errorf("state machine was not started"))
return err
}
n.workItems.ResultEvents().PushBackList(events)
return n.process(exitC, tickC)
}
func (n *Node) RestartProcessing(
exitC <-chan struct{},
tickC <-chan time.Time,
) error {
events, err := processor.RecoverWALForExistingNode(n.processorConfig.WAL, n.runtimeParms())
if err != nil {
n.workErrNotifier.SetExitStatus(nil, errors.Errorf("state machine was not started"))
return err
}
n.workItems.ResultEvents().PushBackList(events)
return n.process(exitC, tickC)
}
func (n *Node) process(exitC <-chan struct{}, tickC <-chan time.Time) error {
var wg sync.WaitGroup
for _, work := range []workFunc{
n.doWALWork,
n.doClientWork,
n.doHashWork, // TODO, spawn more of these
n.doNetWork,
n.doAppWork,
n.doReqStoreWork,
n.doStateMachineWork,
} {
wg.Add(1)
go func(work workFunc) {
wg.Done()
n.doUntilErr(work)
}(work)
}
defer wg.Wait()
var walActionsC, clientActionsC, hashActionsC, netActionsC, appActionsC chan<- *statemachine.ActionList
var reqStoreEventsC, resultEventsC chan<- *statemachine.EventList
for {
select {
case resultEventsC <- n.workItems.ResultEvents():
n.workItems.ClearResultEvents()
resultEventsC = nil
case walActionsC <- n.workItems.WALActions():
n.workItems.ClearWALActions()
walActionsC = nil
case walResultsC := <-n.walResultsC:
n.workItems.AddWALResults(walResultsC)
case clientActionsC <- n.workItems.ClientActions():
n.workItems.ClearClientActions()
clientActionsC = nil
case hashActionsC <- n.workItems.HashActions():
n.workItems.ClearHashActions()
hashActionsC = nil
case netActionsC <- n.workItems.NetActions():
n.workItems.ClearNetActions()
netActionsC = nil
case appActionsC <- n.workItems.AppActions():
n.workItems.ClearAppActions()
appActionsC = nil
case reqStoreEventsC <- n.workItems.ReqStoreEvents():
n.workItems.ClearReqStoreEvents()
reqStoreEventsC = nil
case clientResults := <-n.clientResultsC:
n.workItems.AddClientResults(clientResults)
case hashResults := <-n.hashResultsC:
n.workItems.AddHashResults(hashResults)
case netResults := <-n.netResultsC:
n.workItems.AddNetResults(netResults)
case appResults := <-n.appResultsC:
n.workItems.AddAppResults(appResults)
case reqStoreResults := <-n.reqStoreResultsC:
n.workItems.AddReqStoreResults(reqStoreResults)
case actions := <-n.ResultResultsC:
n.workItems.AddStateMachineResults(actions)
case stepEvents := <-n.replicas.eventC:
// TODO, once request forwarding works, we'll
// need to split this into the req store component
// and 'other' that goes into the result events.
n.workItems.ResultEvents().PushBackList(stepEvents)
case <-n.workErrNotifier.ExitC():
return n.workErrNotifier.Err()
case <-tickC:
n.workItems.ResultEvents().TickElapsed()
case <-exitC:
n.workErrNotifier.Fail(ErrStopped)
}
if resultEventsC == nil && n.workItems.ResultEvents().Len() > 0 {
resultEventsC = n.ResultEventsC
}
if walActionsC == nil && n.workItems.WALActions().Len() > 0 {
walActionsC = n.walActionsC
}
if clientActionsC == nil && n.workItems.ClientActions().Len() > 0 {
clientActionsC = n.clientActionsC
}
if hashActionsC == nil && n.workItems.HashActions().Len() > 0 {
hashActionsC = n.hashActionsC
}
if netActionsC == nil && n.workItems.NetActions().Len() > 0 {
netActionsC = n.netActionsC
}
if appActionsC == nil && n.workItems.AppActions().Len() > 0 {
appActionsC = n.appActionsC
}
if reqStoreEventsC == nil && n.workItems.ReqStoreEvents().Len() > 0 {
reqStoreEventsC = n.reqStoreEventsC
}
}
}
// workErrNotifier is used to synchronize the exit of the assorted worker
// go routines. The first worker to encounter an error should call Fail(err),
// then the other workers will (eventually) read ExitC() to determine that they
// should exit. The worker thread responsible for the state machine _must_
// call SetExitStatus(status, statusErr) before returning.
type workErrNotifier struct {
mutex sync.Mutex
err error
exitC chan struct{}
exitStatus *status.StateMachine
exitStatusErr error
exitStatusC chan struct{}
}
func newWorkErrNotifier() *workErrNotifier {
return &workErrNotifier{
exitC: make(chan struct{}),
exitStatusC: make(chan struct{}),
}
}
func (wen *workErrNotifier) Err() error {
wen.mutex.Lock()
defer wen.mutex.Unlock()
return wen.err
}
func (wen *workErrNotifier) Fail(err error) {
wen.mutex.Lock()
defer wen.mutex.Unlock()
if wen.err != nil {
return
}
wen.err = err
close(wen.exitC)
}
func (wen *workErrNotifier) SetExitStatus(s *status.StateMachine, err error) {
wen.mutex.Lock()
defer wen.mutex.Unlock()
wen.exitStatus = s
wen.exitStatusErr = err
close(wen.exitStatusC)
}
func (wen *workErrNotifier) ExitStatus() (*status.StateMachine, error) {
wen.mutex.Lock()
defer wen.mutex.Unlock()
return wen.exitStatus, wen.exitStatusErr
}
func (wen *workErrNotifier) ExitC() <-chan struct{} {
return wen.exitC
}
func (wen *workErrNotifier) ExitStatusC() <-chan struct{} {
return wen.exitStatusC
}