open-vault/vendor/github.com/hashicorp/raft/commitment.go
Brian Kassouf ed14061578
Raft Storage Backend (#6888)
* Work on raft backend

* Add logstore locally

* Add encryptor and unsealable interfaces

* Add clustering support to raft

* Remove client and handler

* Bootstrap raft on init

* Cleanup raft logic a bit

* More raft work

* Work on TLS config

* More work on bootstrapping

* Fix build

* More work on bootstrapping

* More bootstrapping work

* fix build

* Remove consul dep

* Fix build

* merged oss/master into raft-storage

* Work on bootstrapping

* Get bootstrapping to work

* Clean up FMS and node-id

* Update local node ID logic

* Cleanup node-id change

* Work on snapshotting

* Raft: Add remove peer API (#906)

* Add remove peer API

* Add some comments

* Fix existing snapshotting (#909)

* Raft get peers API (#912)

* Read raft configuration

* address review feedback

* Use the Leadership Transfer API to step-down the active node (#918)

* Raft join and unseal using Shamir keys (#917)

* Raft join using shamir

* Store AEAD instead of master key

* Split the raft join process to answer the challenge after a successful unseal

* get the follower to standby state

* Make unseal work

* minor changes

* Some input checks

* reuse the shamir seal access instead of new default seal access

* refactor joinRaftSendAnswer function

* Synchronously send answer in auto-unseal case

* Address review feedback

* Raft snapshots (#910)

* Fix existing snapshotting

* implement the noop snapshotting

* Add comments and switch log libraries

* add some snapshot tests

* add snapshot test file

* add TODO

* More work on raft snapshotting

* progress on the ConfigStore strategy

* Don't use two buckets

* Update the snapshot store logic to hide the file logic

* Add more backend tests

* Cleanup code a bit

* [WIP] Raft recovery (#938)

* Add recovery functionality

* remove fmt.Printfs

* Fix a few fsm bugs

* Add max size value for raft backend (#942)

* Add max size value for raft backend

* Include physical.ErrValueTooLarge in the message

* Raft snapshot Take/Restore API  (#926)

* Inital work on raft snapshot APIs

* Always redirect snapshot install/download requests

* More work on the snapshot APIs

* Cleanup code a bit

* On restore handle special cases

* Use the seal to encrypt the sha sum file

* Add sealer mechanism and fix some bugs

* Call restore while state lock is held

* Send restore cb trigger through raft log

* Make error messages nicer

* Add test helpers

* Add snapshot test

* Add shamir unseal test

* Add more raft snapshot API tests

* Fix locking

* Change working to initalize

* Add underlying raw object to test cluster core

* Move leaderUUID to core

* Add raft TLS rotation logic (#950)

* Add TLS rotation logic

* Cleanup logic a bit

* Add/Remove from follower state on add/remove peer

* add comments

* Update more comments

* Update request_forwarding_service.proto

* Make sure we populate all nodes in the followerstate obj

* Update times

* Apply review feedback

* Add more raft config setting (#947)

* Add performance config setting

* Add more config options and fix tests

* Test Raft Recovery (#944)

* Test raft recovery

* Leave out a node during recovery

* remove unused struct

* Update physical/raft/snapshot_test.go

* Update physical/raft/snapshot_test.go

* fix vendoring

* Switch to new raft interface

* Remove unused files

* Switch a gogo -> proto instance

* Remove unneeded vault dep in go.sum

* Update helper/testhelpers/testhelpers.go

Co-Authored-By: Calvin Leung Huang <cleung2010@gmail.com>

* Update vault/cluster/cluster.go

* track active key within the keyring itself (#6915)

* track active key within the keyring itself

* lookup and store using the active key ID

* update docstring

* minor refactor

* Small text fixes (#6912)

* Update physical/raft/raft.go

Co-Authored-By: Calvin Leung Huang <cleung2010@gmail.com>

* review feedback

* Move raft logical system into separate file

* Update help text a bit

* Enforce cluster addr is set and use it for raft bootstrapping

* Fix tests

* fix http test panic

* Pull in latest raft-snapshot library

* Add comment
2019-06-20 12:14:58 -07:00

102 lines
3.2 KiB
Go

package raft
import (
"sort"
"sync"
)
// Commitment is used to advance the leader's commit index. The leader and
// replication goroutines report in newly written entries with Match(), and
// this notifies on commitCh when the commit index has advanced.
type commitment struct {
// protects matchIndexes and commitIndex
sync.Mutex
// notified when commitIndex increases
commitCh chan struct{}
// voter ID to log index: the server stores up through this log entry
matchIndexes map[ServerID]uint64
// a quorum stores up through this log entry. monotonically increases.
commitIndex uint64
// the first index of this leader's term: this needs to be replicated to a
// majority of the cluster before this leader may mark anything committed
// (per Raft's commitment rule)
startIndex uint64
}
// newCommitment returns an commitment struct that notifies the provided
// channel when log entries have been committed. A new commitment struct is
// created each time this server becomes leader for a particular term.
// 'configuration' is the servers in the cluster.
// 'startIndex' is the first index created in this term (see
// its description above).
func newCommitment(commitCh chan struct{}, configuration Configuration, startIndex uint64) *commitment {
matchIndexes := make(map[ServerID]uint64)
for _, server := range configuration.Servers {
if server.Suffrage == Voter {
matchIndexes[server.ID] = 0
}
}
return &commitment{
commitCh: commitCh,
matchIndexes: matchIndexes,
commitIndex: 0,
startIndex: startIndex,
}
}
// Called when a new cluster membership configuration is created: it will be
// used to determine commitment from now on. 'configuration' is the servers in
// the cluster.
func (c *commitment) setConfiguration(configuration Configuration) {
c.Lock()
defer c.Unlock()
oldMatchIndexes := c.matchIndexes
c.matchIndexes = make(map[ServerID]uint64)
for _, server := range configuration.Servers {
if server.Suffrage == Voter {
c.matchIndexes[server.ID] = oldMatchIndexes[server.ID] // defaults to 0
}
}
c.recalculate()
}
// Called by leader after commitCh is notified
func (c *commitment) getCommitIndex() uint64 {
c.Lock()
defer c.Unlock()
return c.commitIndex
}
// Match is called once a server completes writing entries to disk: either the
// leader has written the new entry or a follower has replied to an
// AppendEntries RPC. The given server's disk agrees with this server's log up
// through the given index.
func (c *commitment) match(server ServerID, matchIndex uint64) {
c.Lock()
defer c.Unlock()
if prev, hasVote := c.matchIndexes[server]; hasVote && matchIndex > prev {
c.matchIndexes[server] = matchIndex
c.recalculate()
}
}
// Internal helper to calculate new commitIndex from matchIndexes.
// Must be called with lock held.
func (c *commitment) recalculate() {
if len(c.matchIndexes) == 0 {
return
}
matched := make([]uint64, 0, len(c.matchIndexes))
for _, idx := range c.matchIndexes {
matched = append(matched, idx)
}
sort.Sort(uint64Slice(matched))
quorumMatchIndex := matched[(len(matched)-1)/2]
if quorumMatchIndex > c.commitIndex && quorumMatchIndex >= c.startIndex {
c.commitIndex = quorumMatchIndex
asyncNotifyCh(c.commitCh)
}
}