Adding experimental support for a more efficient LogStore implementation (#16176)

* Adding experimental support for a more efficient LogStore implementation

* Adding changelog entry

* Fix go mod tidy issues
This commit is contained in:
Paul Banks 2023-02-08 16:50:22 +00:00 committed by GitHub
parent 1c5ca0da53
commit 50c600f93b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 622 additions and 56 deletions

3
.changelog/16176.txt Normal file
View File

@ -0,0 +1,3 @@
```release-note:improvement
raft: Added experimental `wal` backend for log storage.
```

View File

@ -1479,7 +1479,7 @@ func newConsulConfig(runtimeCfg *config.RuntimeConfig, logger hclog.Logger) (*co
}
cfg.ConfigEntryBootstrap = runtimeCfg.ConfigEntryBootstrap
cfg.RaftBoltDBConfig = runtimeCfg.RaftBoltDBConfig
cfg.LogStoreConfig = runtimeCfg.RaftLogStoreConfig
// Duplicate our own serf config once to make sure that the duplication
// function does not drift.

View File

@ -374,7 +374,6 @@ func (b *builder) build() (rt RuntimeConfig, err error) {
c2.Services = append(c2.Services, *c2.Service)
c2.Service = nil
}
c = Merge(c, c2)
}
@ -1052,6 +1051,7 @@ func (b *builder) build() (rt RuntimeConfig, err error) {
RaftSnapshotThreshold: intVal(c.RaftSnapshotThreshold),
RaftSnapshotInterval: b.durationVal("raft_snapshot_interval", c.RaftSnapshotInterval),
RaftTrailingLogs: intVal(c.RaftTrailingLogs),
RaftLogStoreConfig: b.raftLogStoreConfigVal(&c.RaftLogStore),
ReconnectTimeoutLAN: b.durationVal("reconnect_timeout", c.ReconnectTimeoutLAN),
ReconnectTimeoutWAN: b.durationVal("reconnect_timeout_wan", c.ReconnectTimeoutWAN),
RejoinAfterLeave: boolVal(c.RejoinAfterLeave),
@ -1110,10 +1110,6 @@ func (b *builder) build() (rt RuntimeConfig, err error) {
rt.UseStreamingBackend = boolValWithDefault(c.UseStreamingBackend, true)
if c.RaftBoltDBConfig != nil {
rt.RaftBoltDBConfig = *c.RaftBoltDBConfig
}
if rt.Cache.EntryFetchMaxBurst <= 0 {
return RuntimeConfig{}, fmt.Errorf("cache.entry_fetch_max_burst must be strictly positive, was: %v", rt.Cache.EntryFetchMaxBurst)
}
@ -1384,6 +1380,19 @@ func (b *builder) validate(rt RuntimeConfig) error {
"Consul will refuse to boot with this directory present.\n"+
"See https://www.consul.io/docs/upgrade-specific.html for more information.", mdbPath)
}
// Raft LogStore validation
if rt.RaftLogStoreConfig.Backend != consul.LogStoreBackendBoltDB &&
rt.RaftLogStoreConfig.Backend != consul.LogStoreBackendWAL {
return fmt.Errorf("raft_logstore.backend must be one of '%s' or '%s'",
consul.LogStoreBackendBoltDB, consul.LogStoreBackendWAL)
}
if rt.RaftLogStoreConfig.WAL.SegmentSize < 1024*1024 {
return fmt.Errorf("raft_logstore.wal.segment_size_mb cannot be less than 1MB")
}
if rt.RaftLogStoreConfig.WAL.SegmentSize > 1024*1024*1024 {
return fmt.Errorf("raft_logstore.wal.segment_size_mb cannot be greater than 1024 (1GiB)")
}
}
inuse := map[string]string{}
@ -2707,3 +2716,19 @@ func (b *builder) parsePrefixFilter(telemetry *Telemetry) ([]string, []string) {
return telemetryAllowedPrefixes, telemetryBlockedPrefixes
}
func (b *builder) raftLogStoreConfigVal(raw *RaftLogStoreRaw) consul.RaftLogStoreConfig {
var cfg consul.RaftLogStoreConfig
if raw != nil {
cfg.Backend = stringValWithDefault(raw.Backend, consul.LogStoreBackendBoltDB)
cfg.DisableLogCache = boolVal(raw.DisableLogCache)
cfg.Verification.Enabled = boolVal(raw.Verification.Enabled)
cfg.Verification.Interval = b.durationVal("raft_logstore.verification.interval", raw.Verification.Interval)
cfg.BoltDB.NoFreelistSync = boolVal(raw.BoltDBConfig.NoFreelistSync)
cfg.WAL.SegmentSize = intVal(raw.WALConfig.SegmentSizeMB) * 1024 * 1024
}
return cfg
}

View File

@ -5,8 +5,6 @@ import (
"fmt"
"time"
"github.com/hashicorp/consul/agent/consul"
"github.com/hashicorp/hcl"
"github.com/mitchellh/mapstructure"
@ -250,7 +248,7 @@ type Config struct {
RPC RPC `mapstructure:"rpc" json:"-"`
RaftBoltDBConfig *consul.RaftBoltDBConfig `mapstructure:"raft_boltdb" json:"-"`
RaftLogStore RaftLogStoreRaw `mapstructure:"raft_logstore" json:"raft_logstore,omitempty"`
// UseStreamingBackend instead of blocking queries for service health and
// any other endpoints which support streaming.
@ -921,3 +919,27 @@ type Peering struct {
type XDS struct {
UpdateMaxPerSecond *float64 `mapstructure:"update_max_per_second"`
}
type RaftLogStoreRaw struct {
Backend *string `mapstructure:"backend" json:"backend,omitempty"`
DisableLogCache *bool `mapstructure:"disable_log_cache" json:"disable_log_cache,omitempty"`
Verification RaftLogStoreVerificationRaw `mapstructure:"verification" json:"verification,omitempty"`
BoltDBConfig RaftBoltDBConfigRaw `mapstructure:"boltdb" json:"boltdb,omitempty"`
WALConfig RaftWALConfigRaw `mapstructure:"wal" json:"wal,omitempty"`
}
type RaftLogStoreVerificationRaw struct {
Enabled *bool `mapstructure:"enabled" json:"enabled,omitempty"`
Interval *string `mapstructure:"interval" json:"interval,omitempty"`
}
type RaftBoltDBConfigRaw struct {
NoFreelistSync *bool `mapstructure:"no_freelist_sync" json:"no_freelist_sync,omitempty"`
}
type RaftWALConfigRaw struct {
SegmentSizeMB *int `mapstructure:"segment_size_mb" json:"segment_size_mb,omitempty"`
}

View File

@ -140,7 +140,12 @@ func DefaultSource() Source {
raft_snapshot_threshold = ` + strconv.Itoa(int(cfg.RaftConfig.SnapshotThreshold)) + `
raft_snapshot_interval = "` + cfg.RaftConfig.SnapshotInterval.String() + `"
raft_trailing_logs = ` + strconv.Itoa(int(cfg.RaftConfig.TrailingLogs)) + `
raft_logstore {
backend = "boltdb"
wal {
segment_size_mb = 64
}
}
xds {
update_max_per_second = 250
}

View File

@ -3,6 +3,7 @@ package config
import (
"fmt"
"github.com/hashicorp/consul/agent/consul"
"github.com/hashicorp/consul/types"
)
@ -74,6 +75,9 @@ type DeprecatedConfig struct {
// DEPRECATED(JOIN) - replaced by retry_join_wan
StartJoinAddrsWAN []string `mapstructure:"start_join_wan"`
// DEPRECATED see RaftLogStore
RaftBoltDBConfig *consul.RaftBoltDBConfig `mapstructure:"raft_boltdb" json:"-"`
}
func applyDeprecatedConfig(d *decodeTarget) (Config, []string) {
@ -188,6 +192,13 @@ func applyDeprecatedConfig(d *decodeTarget) (Config, []string) {
warns = append(warns, deprecationWarning("start_join_wan", "retry_join_wan"))
}
if dep.RaftBoltDBConfig != nil {
if d.Config.RaftLogStore.BoltDBConfig.NoFreelistSync == nil {
d.Config.RaftLogStore.BoltDBConfig.NoFreelistSync = &dep.RaftBoltDBConfig.NoFreelistSync
}
warns = append(warns, deprecationWarning("raft_boltdb", "raft_logstore.boltdb"))
}
warns = append(warns, applyDeprecatedTLSConfig(dep, &d.Config)...)
return d.Config, warns

View File

@ -42,6 +42,10 @@ verify_incoming_rpc = false
verify_outgoing = true
verify_server_hostname = true
tls_prefer_server_cipher_suites = true
raft_boltdb {
NoFreelistSync = true
}
`},
}
patchLoadOptsShims(&opts)
@ -70,6 +74,7 @@ tls_prefer_server_cipher_suites = true
deprecationWarning("verify_outgoing", "tls.defaults.verify_outgoing"),
deprecationWarning("verify_server_hostname", "tls.internal_rpc.verify_server_hostname"),
"The 'tls_prefer_server_cipher_suites' field is deprecated and will be ignored.",
deprecationWarning("raft_boltdb", "raft_logstore.boltdb"),
}
require.ElementsMatch(t, expectWarns, result.Warnings)
// Ideally this would compare against the entire result.RuntimeConfig, but
@ -102,6 +107,7 @@ tls_prefer_server_cipher_suites = true
require.True(t, rt.TLS.InternalRPC.VerifyOutgoing)
require.True(t, rt.TLS.HTTPS.VerifyOutgoing)
require.True(t, rt.TLS.InternalRPC.VerifyServerHostname)
require.True(t, rt.RaftLogStoreConfig.BoltDB.NoFreelistSync)
}
func TestLoad_DeprecatedConfig_ACLReplication(t *testing.T) {

View File

@ -978,7 +978,7 @@ type RuntimeConfig struct {
// hcl: raft_trailing_logs = int
RaftTrailingLogs int
RaftBoltDBConfig consul.RaftBoltDBConfig
RaftLogStoreConfig consul.RaftLogStoreConfig
// ReconnectTimeoutLAN specifies the amount of time to wait to reconnect with
// another agent before deciding it's permanently gone. This can be used to

View File

@ -5754,6 +5754,143 @@ func TestLoad_IntegrationWithFlags(t *testing.T) {
rt.TLS.GRPC.UseAutoCert = false
},
})
run(t, testCase{
desc: "logstore defaults",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{``},
hcl: []string{``},
expected: func(rt *RuntimeConfig) {
rt.DataDir = dataDir
rt.RaftLogStoreConfig.Backend = consul.LogStoreBackendBoltDB
rt.RaftLogStoreConfig.WAL.SegmentSize = 64 * 1024 * 1024
},
})
run(t, testCase{
// this was a bug in the initial config commit. Specifying part of this
// stanza should still result in sensible defaults for the other parts.
desc: "wal defaults",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{`{
"raft_logstore": {
"backend": "boltdb"
}
}`},
hcl: []string{`
raft_logstore {
backend = "boltdb"
}
`},
expected: func(rt *RuntimeConfig) {
rt.DataDir = dataDir
rt.RaftLogStoreConfig.Backend = consul.LogStoreBackendBoltDB
rt.RaftLogStoreConfig.WAL.SegmentSize = 64 * 1024 * 1024
},
})
run(t, testCase{
desc: "wal segment size lower bound",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{`
{
"server": true,
"raft_logstore": {
"wal":{
"segment_size_mb": 0
}
}
}`},
hcl: []string{`
server = true
raft_logstore {
wal {
segment_size_mb = 0
}
}`},
expectedErr: "raft_logstore.wal.segment_size_mb cannot be less than",
})
run(t, testCase{
desc: "wal segment size upper bound",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{`
{
"server": true,
"raft_logstore": {
"wal":{
"segment_size_mb": 1025
}
}
}`},
hcl: []string{`
server = true
raft_logstore {
wal {
segment_size_mb = 1025
}
}`},
expectedErr: "raft_logstore.wal.segment_size_mb cannot be greater than",
})
run(t, testCase{
desc: "valid logstore backend",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{`
{
"server": true,
"raft_logstore": {
"backend": "thecloud"
}
}`},
hcl: []string{`
server = true
raft_logstore {
backend = "thecloud"
}`},
expectedErr: "raft_logstore.backend must be one of 'boltdb' or 'wal'",
})
run(t, testCase{
desc: "raft_logstore merging",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{
// File 1 has logstore info
`{
"raft_logstore": {
"backend": "wal"
}
}`,
// File 2 doesn't have anything for logstore
`{
"enable_debug": true
}`,
},
hcl: []string{
// File 1 has logstore info
`
raft_logstore {
backend = "wal"
}`,
// File 2 doesn't have anything for logstore
`
enable_debug = true
`,
},
expected: func(rt *RuntimeConfig) {
rt.DataDir = dataDir
// The logstore settings from first file should not be overridden by a
// later file with nothing to say about logstores!
rt.RaftLogStoreConfig.Backend = consul.LogStoreBackendWAL
rt.EnableDebug = true
},
})
}
func (tc testCase) run(format string, dataDir string) func(t *testing.T) {
@ -6627,8 +6764,17 @@ func TestLoad_FullConfig(t *testing.T) {
"args": []interface{}{"dltjDJ2a", "flEa7C2d"},
},
},
XDSUpdateRateLimit: 9526.2,
RaftBoltDBConfig: consul.RaftBoltDBConfig{NoFreelistSync: true},
XDSUpdateRateLimit: 9526.2,
RaftLogStoreConfig: consul.RaftLogStoreConfig{
Backend: consul.LogStoreBackendWAL,
DisableLogCache: true,
Verification: consul.RaftLogStoreVerificationConfig{
Enabled: true,
Interval: 12345 * time.Second,
},
BoltDB: consul.RaftBoltDBConfig{NoFreelistSync: true},
WAL: consul.WALConfig{SegmentSize: 15 * 1024 * 1024},
},
AutoReloadConfigCoalesceInterval: 1 * time.Second,
}
entFullRuntimeConfig(expected)

View File

@ -16,9 +16,9 @@
"ACLTokens": {
"ACLAgentRecoveryToken": "hidden",
"ACLAgentToken": "hidden",
"ACLConfigFileRegistrationToken": "hidden",
"ACLDefaultToken": "hidden",
"ACLReplicationToken": "hidden",
"ACLConfigFileRegistrationToken": "hidden",
"DataDir": "",
"EnablePersistence": false,
"EnterpriseConfig": {}
@ -268,8 +268,19 @@
"RPCMaxConnsPerClient": 0,
"RPCProtocol": 0,
"RPCRateLimit": 0,
"RaftBoltDBConfig": {
"NoFreelistSync": false
"RaftLogStoreConfig": {
"Backend": "",
"BoltDB": {
"NoFreelistSync": false
},
"DisableLogCache": false,
"Verification": {
"Enabled": false,
"Interval": "0s"
},
"WAL": {
"SegmentSize": 0
}
},
"RaftProtocol": 3,
"RaftSnapshotInterval": "0s",
@ -487,4 +498,4 @@
"VersionPrerelease": "",
"Watches": [],
"XDSUpdateRateLimit": 0
}
}

View File

@ -353,8 +353,19 @@ raft_protocol = 3
raft_snapshot_threshold = 16384
raft_snapshot_interval = "30s"
raft_trailing_logs = 83749
raft_boltdb {
NoFreelistSync = true
raft_logstore {
backend = "wal"
disable_log_cache = true
verification {
enabled = true
interval = "12345s"
}
boltdb {
no_freelist_sync = true
}
wal {
segment_size_mb = 15
}
}
read_replica = true
reconnect_timeout = "23739s"

View File

@ -406,8 +406,19 @@
"raft_snapshot_threshold": 16384,
"raft_snapshot_interval": "30s",
"raft_trailing_logs": 83749,
"raft_boltdb": {
"NoFreelistSync": true
"raft_logstore": {
"backend" : "wal",
"disable_log_cache": true,
"verification": {
"enabled": true,
"interval":"12345s"
},
"boltdb": {
"no_freelist_sync": true
},
"wal": {
"segment_size_mb": 15
}
},
"read_replica": true,
"reconnect_timeout": "23739s",

View File

@ -34,6 +34,11 @@ const (
// MaxRaftMultiplier is a fairly arbitrary upper bound that limits the
// amount of performance detuning that's possible.
MaxRaftMultiplier uint = 10
// LogStoreBackend* are well-known string values used to configure different
// log store backends.
LogStoreBackendBoltDB = "boltdb"
LogStoreBackendWAL = "wal"
)
var (
@ -424,7 +429,7 @@ type Config struct {
RPCConfig RPCConfig
RaftBoltDBConfig RaftBoltDBConfig
LogStoreConfig RaftLogStoreConfig
// PeeringEnabled enables cluster peering.
PeeringEnabled bool
@ -668,6 +673,23 @@ type ReloadableConfig struct {
ElectionTimeout time.Duration
}
type RaftLogStoreConfig struct {
Backend string
DisableLogCache bool
Verification RaftLogStoreVerificationConfig
BoltDB RaftBoltDBConfig
WAL WALConfig
}
type RaftLogStoreVerificationConfig struct {
Enabled bool
Interval time.Duration
}
type RaftBoltDBConfig struct {
NoFreelistSync bool
}
type WALConfig struct {
SegmentSize int
}

View File

@ -114,8 +114,10 @@ func NewFromDeps(deps Deps) *FSM {
return fsm
}
func (c *FSM) ChunkingFSM() *raftchunking.ChunkingFSM {
return c.chunker
func (c *FSM) ChunkingFSM() raft.FSM {
// Wrap the chunker in a shim. This is not a ChunkingFSM any more but the only
// caller of this passes it directly to Raft as a raft.FSM.
return &logVerificationChunkingShim{chunker: c.chunker}
}
// State is used to return a handle to the current state
@ -129,6 +131,11 @@ func (c *FSM) Apply(log *raft.Log) interface{} {
buf := log.Data
msgType := structs.MessageType(buf[0])
// This is tricky stuff. We no longer let the ChunkingFSM wrap us completely
// because Chunking FSM doesn't know how to handle raft log verification
// checkpoints properly. So instead we have to be extra careful to correctly
// call into the chunking FSM when we need it.
// Check if this message type should be ignored when unknown. This is
// used so that new commands can be added with developer control if older
// versions can safely ignore the command, or if they should crash.

View File

@ -0,0 +1,63 @@
package fsm
import (
"bytes"
"encoding/binary"
"io"
"github.com/hashicorp/go-raftchunking"
"github.com/hashicorp/raft"
"github.com/hashicorp/raft-wal/verifier"
"github.com/hashicorp/consul/agent/structs"
)
type logVerificationChunkingShim struct {
chunker *raftchunking.ChunkingFSM
}
var logVerifierMagicBytes [8]byte
func init() {
binary.LittleEndian.PutUint64(logVerifierMagicBytes[:], verifier.ExtensionMagicPrefix)
}
// Apply implements raft.FSM.
func (s *logVerificationChunkingShim) Apply(l *raft.Log) interface{} {
// This is a hack because raftchunking doesn't play nicely with lower-level
// usage of Extensions field like we need for LogStore verification. We might
// change that instead but just seeing if I can get this to work here without
// upstream changes for now.
// We rely on the way we encode a checkpoint message being distinguishable
// from any valid chunked log entry. The type byte alone or the fact there is
// only one byte of data is not quite enough because it's just possible that
// chunking might split a larger log such that its final chunk was just a
// single byte, and if so there is a 1 in 256 chance it collides with our type
// byte! But we specially chose a magic value for verifier.LogStore to use
// that would never be the first 8 bytes of a valid proto encoding. See the
// docs on that value for more detail on why not. Note the data length for a
// checkpoint is actually 2 because msgpack encodes the nil slice as a typed
// nil byte (0xc0).
if len(l.Data) == 2 &&
structs.MessageType(l.Data[0]) == (structs.RaftLogVerifierCheckpoint|structs.IgnoreUnknownTypeFlag) &&
len(l.Extensions) > 8 &&
bytes.Equal(logVerifierMagicBytes[:], l.Extensions[0:8]) {
// Handle the checkpoint here since the lower level FSM doesn't know
// anything about it! The LogStore has already done what we need, we just
// need to return the index so that the caller can know which index the
// checkpoint ended up at.
return l.Index
}
return s.chunker.Apply(l)
}
// Snapshot implements raft.FSM
func (s *logVerificationChunkingShim) Snapshot() (raft.FSMSnapshot, error) {
return s.chunker.Snapshot()
}
// Restore implements raft.FSM
func (s *logVerificationChunkingShim) Restore(snapshot io.ReadCloser) error {
return s.chunker.Restore(snapshot)
}

View File

@ -334,6 +334,10 @@ func (s *Server) establishLeadership(ctx context.Context) error {
s.setConsistentReadReady()
if s.config.LogStoreConfig.Verification.Enabled {
s.startLogVerification(ctx)
}
s.logger.Debug("successfully established leadership", "duration", time.Since(start))
return nil
}
@ -341,6 +345,9 @@ func (s *Server) establishLeadership(ctx context.Context) error {
// revokeLeadership is invoked once we step down as leader.
// This is used to cleanup any state that may be specific to a leader.
func (s *Server) revokeLeadership() {
s.stopLogVerification()
// Disable the tombstone GC, since it is only useful as a leader
s.tombstoneGC.SetEnabled(false)

View File

@ -0,0 +1,48 @@
package consul
import (
"context"
"time"
"github.com/hashicorp/consul/agent/structs"
)
func (s *Server) startLogVerification(ctx context.Context) error {
return s.leaderRoutineManager.Start(ctx, raftLogVerifierRoutineName, s.runLogVerification)
}
func (s *Server) stopLogVerification() {
s.leaderRoutineManager.Stop(raftLogVerifierRoutineName)
}
func (s *Server) runLogVerification(ctx context.Context) error {
// This shouldn't be possible but bit of a safety check
if !s.config.LogStoreConfig.Verification.Enabled ||
s.config.LogStoreConfig.Verification.Interval == 0 {
return nil
}
ticker := time.NewTicker(s.config.LogStoreConfig.Verification.Interval)
defer ticker.Stop()
logger := s.logger.Named("raft.logstore.verifier")
for {
select {
case <-ticker.C:
// Attempt to send a checkpoint message
typ := structs.RaftLogVerifierCheckpoint | structs.IgnoreUnknownTypeFlag
raw, err := s.raftApplyMsgpack(typ, nil)
if err != nil {
logger.Error("sending verification checkpoint failed", "err", err)
} else {
index, ok := raw.(uint64)
if !ok {
index = 0
}
logger.Debug("sent verification checkpoint", "index", int64(index))
}
case <-ctx.Done():
return nil
}
}
}

View File

@ -23,6 +23,9 @@ import (
"github.com/hashicorp/raft"
autopilot "github.com/hashicorp/raft-autopilot"
raftboltdb "github.com/hashicorp/raft-boltdb/v2"
raftwal "github.com/hashicorp/raft-wal"
walmetrics "github.com/hashicorp/raft-wal/metrics"
"github.com/hashicorp/raft-wal/verifier"
"github.com/hashicorp/serf/serf"
"go.etcd.io/bbolt"
"golang.org/x/time/rate"
@ -134,6 +137,7 @@ const (
peeringStreamsRoutineName = "streaming peering resources"
peeringDeletionRoutineName = "peering deferred deletion"
peeringStreamsMetricsRoutineName = "metrics for streaming peering resources"
raftLogVerifierRoutineName = "raft log verifier"
)
var (
@ -145,6 +149,13 @@ const (
PoolKindSegment = "segment"
)
// raftStore combines LogStore and io.Closer since we need both but have
// multiple LogStore implementations that need closing too.
type raftStore interface {
raft.LogStore
io.Closer
}
const requestLimitsBurstMultiplier = 10
// Server is Consul server which manages the service discovery,
@ -228,7 +239,7 @@ type Server struct {
// the state directly.
raft *raft.Raft
raftLayer *RaftLayer
raftStore *raftboltdb.BoltStore
raftStore raftStore
raftTransport *raft.NetworkTransport
raftInmem *raft.InmemStore
@ -964,24 +975,68 @@ func (s *Server) setupRaft() error {
return err
}
// Create the backend raft store for logs and stable storage.
store, err := raftboltdb.New(raftboltdb.Options{
BoltOptions: &bbolt.Options{
NoFreelistSync: s.config.RaftBoltDBConfig.NoFreelistSync,
},
Path: filepath.Join(path, "raft.db"),
})
boltDBFile := filepath.Join(path, "raft.db")
boltFileExists, err := fileExists(boltDBFile)
if err != nil {
return err
return fmt.Errorf("failed trying to see if raft.db exists not sure how to continue: %w", err)
}
s.raftStore = store
stable = store
// start publishing boltdb metrics
go store.RunMetrics(&lib.StopChannelContext{StopCh: s.shutdownCh}, 0)
// Only use WAL if there is no existing raft.db, even if it's enabled.
if s.config.LogStoreConfig.Backend == LogStoreBackendWAL && !boltFileExists {
walDir := filepath.Join(path, "wal")
if err := os.MkdirAll(walDir, 0755); err != nil {
return err
}
mc := walmetrics.NewGoMetricsCollector([]string{"raft", "wal"}, nil, nil)
wal, err := raftwal.Open(walDir,
raftwal.WithSegmentSize(s.config.LogStoreConfig.WAL.SegmentSize),
raftwal.WithMetricsCollector(mc),
)
if err != nil {
return fmt.Errorf("fail to open write-ahead-log: %w", err)
}
s.raftStore = wal
log = wal
stable = wal
} else {
if s.config.LogStoreConfig.Backend == LogStoreBackendWAL {
// User configured the new storage, but still has old raft.db. Warn
// them!
s.logger.Warn("BoltDB file raft.db found, IGNORING raft_logstore.backend which is set to 'wal'")
}
// Create the backend raft store for logs and stable storage.
store, err := raftboltdb.New(raftboltdb.Options{
BoltOptions: &bbolt.Options{
NoFreelistSync: s.config.LogStoreConfig.BoltDB.NoFreelistSync,
},
Path: boltDBFile,
})
if err != nil {
return err
}
s.raftStore = store
log = store
stable = store
// start publishing boltdb metrics
go store.RunMetrics(&lib.StopChannelContext{StopCh: s.shutdownCh}, 0)
}
// See if log verification is enabled
if s.config.LogStoreConfig.Verification.Enabled {
mc := walmetrics.NewGoMetricsCollector([]string{"raft", "logstore", "verifier"}, nil, nil)
reportFn := makeLogVerifyReportFn(s.logger.Named("raft.logstore.verifier"))
verifier := verifier.NewLogStore(log, isLogVerifyCheckpoint, reportFn, mc)
s.raftStore = verifier
log = verifier
}
// Wrap the store in a LogCache to improve performance.
cacheStore, err := raft.NewLogCache(raftLogCacheSize, store)
cacheStore, err := raft.NewLogCache(raftLogCacheSize, log)
if err != nil {
return err
}
@ -1847,6 +1902,20 @@ func (s *Server) hcpServerStatus(deps Deps) hcp.StatusCallback {
}
}
func fileExists(name string) (bool, error) {
_, err := os.Stat(name)
if err == nil {
// File exists!
return true, nil
}
if errors.Is(err, os.ErrNotExist) {
return false, nil
}
// We hit some other error trying to stat the file which leaves us in an
// unknown state so we can't proceed.
return false, err
}
func ConfiguredIncomingRPCLimiter(ctx context.Context, serverLogger hclog.InterceptLogger, consulCfg *Config) *rpcRate.Handler {
mlCfg := &multilimiter.Config{ReconcileCheckLimit: 30 * time.Second, ReconcileCheckInterval: time.Second}
limitsConfig := &RequestLimits{

View File

@ -0,0 +1,81 @@
package consul
import (
"errors"
"fmt"
"github.com/hashicorp/go-hclog"
"github.com/hashicorp/raft"
"github.com/hashicorp/raft-wal/verifier"
"github.com/hashicorp/consul/agent/structs"
)
var _ verifier.IsCheckpointFn = isLogVerifyCheckpoint
// isLogVerifyCheckpoint is the verifier.IsCheckpointFn that can decode our raft logs for
// their type.
func isLogVerifyCheckpoint(l *raft.Log) (bool, error) {
if len(l.Data) < 1 {
// Shouldn't be possible! But no need to make it an error if it wasn't one
// before.
return false, nil
}
// Allow for the "ignore missing" bit to be set.
typ := structs.MessageType(l.Data[0])
if typ&structs.IgnoreUnknownTypeFlag == structs.IgnoreUnknownTypeFlag {
typ &= ^structs.IgnoreUnknownTypeFlag
}
return typ == structs.RaftLogVerifierCheckpoint, nil
}
func makeLogVerifyReportFn(logger hclog.Logger) verifier.ReportFn {
return func(r verifier.VerificationReport) {
if r.SkippedRange != nil {
logger.Warn("verification skipped range, consider decreasing validation interval if this is frequent",
"rangeStart", int64(r.SkippedRange.Start),
"rangeEnd", int64(r.SkippedRange.End),
)
}
l2 := logger.With(
"rangeStart", int64(r.Range.Start),
"rangeEnd", int64(r.Range.End),
"leaderChecksum", fmt.Sprintf("%08x", r.ExpectedSum),
"elapsed", r.Elapsed,
)
if r.Err == nil {
l2.Info("verification checksum OK",
"readChecksum", fmt.Sprintf("%08x", r.ReadSum),
)
return
}
if r.Err == verifier.ErrRangeMismatch {
l2.Warn("verification checksum skipped as we don't have all logs in range")
return
}
var csErr verifier.ErrChecksumMismatch
if errors.As(r.Err, &csErr) {
if r.WrittenSum > 0 && r.WrittenSum != r.ExpectedSum {
// The failure occurred before the follower wrote to the log so it
// must be corrupted in flight from the leader!
l2.Info("verification checksum FAILED: in-flight corruption",
"followerWriteChecksum", fmt.Sprintf("%08x", r.WrittenSum),
"readChecksum", fmt.Sprintf("%08x", r.ReadSum),
)
} else {
l2.Info("verification checksum FAILED: storage corruption",
"followerWriteChecksum", fmt.Sprintf("%08x", r.WrittenSum),
"readChecksum", fmt.Sprintf("%08x", r.ReadSum),
)
}
return
}
// Some other unknown error occurred
l2.Error(r.Err.Error())
}
}

View File

@ -83,6 +83,7 @@ const (
PeeringTrustBundleWriteType = 38
PeeringTrustBundleDeleteType = 39
PeeringSecretsWriteType = 40
RaftLogVerifierCheckpoint = 41 // Only used for log verifier, no-op on FSM.
)
const (
@ -149,6 +150,7 @@ var requestTypeStrings = map[MessageType]string{
PeeringTrustBundleWriteType: "PeeringTrustBundle",
PeeringTrustBundleDeleteType: "PeeringTrustBundleDelete",
PeeringSecretsWriteType: "PeeringSecret",
RaftLogVerifierCheckpoint: "RaftLogVerifierCheckpoint",
}
const (

View File

@ -105,8 +105,7 @@ struct they have their own struct called `Config` in `agent/consul/config.go`.
- [ ] Do all of the steps in [Adding a Simple Config
Field For Client Agents](#adding-a-simple-config-field-for-client-agents).
- [ ] Add the new field to Config struct in `agent/consul/config.go`
- [ ] Add code to set the values from the `RuntimeConfig` in the confusingly
named `consulConfig` method in `agent/agent.go`
- [ ] Add code to set the values from the `RuntimeConfig` in `newConsulConfig` method in `agent/agent.go`
- [ ] **If needed**, add a test to `agent_test.go` if there is some non-trivial
behavior in the code you added in the previous step. We tend not to test
simple assignments from one to the other since these are typically caught by

16
go.mod
View File

@ -8,6 +8,10 @@ replace (
github.com/hashicorp/consul/proto-public => ./proto-public
github.com/hashicorp/consul/sdk => ./sdk
github.com/hashicorp/consul/troubleshoot => ./troubleshoot
// pinning this x/time version because consul-k8s acceptance tests fail
// with client rate limiting issues with newer versions of this package.
// This is tracked in NET-2284
golang.org/x/time => golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e
)
exclude (
@ -65,6 +69,7 @@ require (
github.com/hashicorp/raft v1.3.11
github.com/hashicorp/raft-autopilot v0.1.6
github.com/hashicorp/raft-boltdb/v2 v2.2.2
github.com/hashicorp/raft-wal v0.2.4
github.com/hashicorp/serf v0.10.1
github.com/hashicorp/vault/api v1.8.2
github.com/hashicorp/vault/api/auth/gcp v0.3.0
@ -88,14 +93,14 @@ require (
github.com/ryanuber/columnize v2.1.2+incompatible
github.com/shirou/gopsutil/v3 v3.22.8
github.com/stretchr/testify v1.8.0
go.etcd.io/bbolt v1.3.5
go.etcd.io/bbolt v1.3.6
go.uber.org/goleak v1.1.10
golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d
golang.org/x/net v0.4.0
golang.org/x/oauth2 v0.0.0-20220909003341-f21342109be1
golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4
golang.org/x/sys v0.3.0
golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e
golang.org/x/time v0.1.0
google.golang.org/genproto v0.0.0-20220921223823-23cae91e6737
google.golang.org/grpc v1.49.0
google.golang.org/protobuf v1.28.1
@ -124,6 +129,7 @@ require (
github.com/PuerkitoBio/purell v1.1.1 // indirect
github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578 // indirect
github.com/asaskevich/govalidator v0.0.0-20210307081110-f21760c49a8d // indirect
github.com/benbjohnson/immutable v0.4.0 // indirect
github.com/beorn7/perks v1.0.1 // indirect
github.com/bgentry/speakeasy v0.1.0 // indirect
github.com/boltdb/bolt v1.3.1 // indirect
@ -133,6 +139,9 @@ require (
github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible // indirect
github.com/circonus-labs/circonusllhist v0.1.3 // indirect
github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1 // indirect
github.com/coreos/etcd v3.3.27+incompatible // indirect
github.com/coreos/go-systemd v0.0.0-20191104093116-d3cd4ed1dbcf // indirect
github.com/coreos/pkg v0.0.0-20220810130054-c7d1c02cb6cf // indirect
github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect
github.com/denverdino/aliyungo v0.0.0-20170926055100-d3308649c661 // indirect
github.com/digitalocean/godo v1.10.0 // indirect
@ -169,7 +178,6 @@ require (
github.com/hashicorp/go-secure-stdlib/strutil v0.1.2 // indirect
github.com/hashicorp/mdns v1.0.4 // indirect
github.com/hashicorp/net-rpc-msgpackrpc/v2 v2.0.0 // indirect
github.com/hashicorp/raft-boltdb v0.0.0-20211202195631-7d34b9fb3f42 // indirect
github.com/hashicorp/vic v1.5.1-0.20190403131502-bbfe86ec9443 // indirect
github.com/jmespath/go-jmespath v0.4.0 // indirect
github.com/josharian/intern v1.0.0 // indirect
@ -202,6 +210,7 @@ require (
github.com/renier/xmlrpc v0.0.0-20170708154548-ce4a1a486c03 // indirect
github.com/ryanuber/go-glob v1.0.0 // indirect
github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 // indirect
github.com/segmentio/fasthash v1.0.3 // indirect
github.com/sirupsen/logrus v1.6.0 // indirect
github.com/skratchdot/open-golang v0.0.0-20200116055534-eef842397966 // indirect
github.com/softlayer/softlayer-go v0.0.0-20180806151055-260589d94c7d // indirect
@ -217,6 +226,7 @@ require (
go.opencensus.io v0.23.0 // indirect
go.opentelemetry.io/proto/otlp v0.7.0 // indirect
go.uber.org/atomic v1.9.0 // indirect
golang.org/x/exp v0.0.0-20220827204233-334a2380cb91 // indirect
golang.org/x/lint v0.0.0-20210508222113-6edffad5e616 // indirect
golang.org/x/term v0.3.0 // indirect
golang.org/x/text v0.5.0 // indirect

33
go.sum
View File

@ -141,7 +141,6 @@ github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hC
github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8=
github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY=
github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878/go.mod h1:3AMJUQhVx52RsWOnlkpikZr01T/yAVN2gn0861vByNg=
github.com/armon/go-metrics v0.3.8/go.mod h1:4O98XIr/9W0sxpJ8UaYkvjk10Iff7SnFrb4QAOwNTFc=
github.com/armon/go-metrics v0.3.9/go.mod h1:4O98XIr/9W0sxpJ8UaYkvjk10Iff7SnFrb4QAOwNTFc=
github.com/armon/go-metrics v0.3.10 h1:FR+drcQStOe+32sYyJYyZ7FIdgoGGBnwLl+flodp8Uo=
github.com/armon/go-metrics v0.3.10/go.mod h1:4O98XIr/9W0sxpJ8UaYkvjk10Iff7SnFrb4QAOwNTFc=
@ -158,6 +157,8 @@ github.com/aws/aws-sdk-go v1.30.27/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve
github.com/aws/aws-sdk-go v1.42.34 h1:fqGAiKmCSRY1rEa4G9VqgkKKbNmLKYq5dKmLtQkvYi8=
github.com/aws/aws-sdk-go v1.42.34/go.mod h1:OGr6lGMAKGlG9CVrYnWYDKIyb829c6EVBRjxqjmPepc=
github.com/baiyubin/aliyun-sts-go-sdk v0.0.0-20180326062324-cfa1a18b161f/go.mod h1:AuiFmCCPBSrqvVMvuqFuk0qogytodnVFVSN5CeJB8Gc=
github.com/benbjohnson/immutable v0.4.0 h1:CTqXbEerYso8YzVPxmWxh2gnoRQbbB9X1quUC8+vGZA=
github.com/benbjohnson/immutable v0.4.0/go.mod h1:iAr8OjJGLnLmVUr9MZ/rz4PWUy6Ouc2JLYuMArmvAJM=
github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
@ -202,6 +203,8 @@ github.com/coredns/coredns v1.6.6/go.mod h1:Bdcnka9HmKGYj12ZIDF3lpQSfDHSsMc85Wj9
github.com/coredns/federation v0.0.0-20190818181423-e032b096babe/go.mod h1:MoqTEFX8GlnKkyq8eBCF94VzkNAOgjdlCJ+Pz/oCLPk=
github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk=
github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
github.com/coreos/etcd v3.3.27+incompatible h1:QIudLb9KeBsE5zyYxd1mjzRSkzLg9Wf9QlRwFgd6oTA=
github.com/coreos/etcd v3.3.27+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk=
github.com/coreos/go-oidc v2.1.0+incompatible h1:sdJrfw8akMnCuUlaZU3tE/uYXFgfqom8DBE9so9EBsM=
github.com/coreos/go-oidc v2.1.0+incompatible/go.mod h1:CgnwVTmzoESiwO9qyAFEMiHoZ1nMCKZlZ9V6mm3/LKc=
@ -209,8 +212,12 @@ github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3Ee
github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
github.com/coreos/go-systemd v0.0.0-20190212144455-93d5ec2c7f76/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
github.com/coreos/go-systemd v0.0.0-20191104093116-d3cd4ed1dbcf h1:iW4rZ826su+pqaw19uhpSCzhj44qo35pNgKFGqzDKkU=
github.com/coreos/go-systemd v0.0.0-20191104093116-d3cd4ed1dbcf/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
github.com/coreos/pkg v0.0.0-20220810130054-c7d1c02cb6cf h1:GOPo6vn/vTN+3IwZBvXX0y5doJfSC7My0cdzelyOCsQ=
github.com/coreos/pkg v0.0.0-20220810130054-c7d1c02cb6cf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
github.com/cpu/goacmedns v0.0.1/go.mod h1:sesf/pNnCYwUevQEQfEwY0Y3DydlQWSGZbaMElOWxok=
github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE=
github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU=
@ -606,10 +613,11 @@ github.com/hashicorp/raft-autopilot v0.1.6 h1:C1q3RNF2FfXNZfHWbvVAu0QixaQK8K5pX4
github.com/hashicorp/raft-autopilot v0.1.6/go.mod h1:Af4jZBwaNOI+tXfIqIdbcAnh/UyyqIMj/pOISIfhArw=
github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea/go.mod h1:pNv7Wc3ycL6F5oOWn+tPGo2gWD4a5X+yp/ntwdKLjRk=
github.com/hashicorp/raft-boltdb v0.0.0-20210409134258-03c10cc3d4ea/go.mod h1:qRd6nFJYYS6Iqnc/8HcUmko2/2Gw8qTFEmxDLii6W5I=
github.com/hashicorp/raft-boltdb v0.0.0-20211202195631-7d34b9fb3f42 h1:Ye8SofeDHJzu9xvvaMmpMkqHELWW7rTcXwdUR0CWW48=
github.com/hashicorp/raft-boltdb v0.0.0-20211202195631-7d34b9fb3f42/go.mod h1:wcXL8otVu5cpJVLjcmq7pmfdRCdaP+xnvu7WQcKJAhs=
github.com/hashicorp/raft-boltdb v0.0.0-20220329195025-15018e9b97e0 h1:CO8dBMLH6dvE1jTn/30ZZw3iuPsNfajshWoJTnVc5cc=
github.com/hashicorp/raft-boltdb/v2 v2.2.2 h1:rlkPtOllgIcKLxVT4nutqlTH2NRFn+tO1wwZk/4Dxqw=
github.com/hashicorp/raft-boltdb/v2 v2.2.2/go.mod h1:N8YgaZgNJLpZC+h+by7vDu5rzsRgONThTEeUS3zWbfY=
github.com/hashicorp/raft-wal v0.2.4 h1:Ke0ytMj8XyOVKQqFDmmgs/6hqkTJg0b/GO2a2XQBZ6A=
github.com/hashicorp/raft-wal v0.2.4/go.mod h1:JQ/4RbnKFi5Q/4rA73CekaYtHCJhU7qM7AQ4X5Y6q4M=
github.com/hashicorp/serf v0.10.1 h1:Z1H2J60yRKvfDYAOZLd2MU0ND4AH/WDz7xYHDWQsIPY=
github.com/hashicorp/serf v0.10.1/go.mod h1:yL2t6BqATOLGc5HF7qbFkTfXoPIY0WZdWHfEvMqbG+4=
github.com/hashicorp/vault/api v1.8.0/go.mod h1:uJrw6D3y9Rv7hhmS17JQC50jbPDAZdjZoTtrCCxxs7E=
@ -689,8 +697,8 @@ github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFB
github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0=
github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk=
github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE=
github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
@ -909,8 +917,8 @@ github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6L
github.com/rogpeppe/go-internal v1.1.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
github.com/rogpeppe/go-internal v1.2.2/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k=
github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc=
github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8=
github.com/rs/zerolog v1.4.0/go.mod h1:YbFCdg8HfsridGWAh22vktObvhZbQsZXe4/zB0OKkWU=
github.com/russross/blackfriday v0.0.0-20170610170232-067529f716f4/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g=
github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g=
@ -927,6 +935,8 @@ github.com/sean-/conswriter v0.0.0-20180208195008-f5ae3917a627/go.mod h1:7zjs06q
github.com/sean-/pager v0.0.0-20180208200047-666be9bf53b5/go.mod h1:BeybITEsBEg6qbIiqJ6/Bqeq25bCLbL7YFmpaFfJDuM=
github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 h1:nn5Wsu0esKSJiIVhscUtVbo7ada43DJhG55ua/hjS5I=
github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc=
github.com/segmentio/fasthash v1.0.3 h1:EI9+KE1EwvMLBWwjpRDc+fEM+prwxDYbslddQGtrmhM=
github.com/segmentio/fasthash v1.0.3/go.mod h1:waKX8l2N8yckOgmSsXJi7x1ZfdKZ4x7KRMzBtS3oedY=
github.com/shirou/gopsutil/v3 v3.22.8 h1:a4s3hXogo5mE2PfdfJIonDbstO/P+9JszdfhAHSzD9Y=
github.com/shirou/gopsutil/v3 v3.22.8/go.mod h1:s648gW4IywYzUfE/KjXxUsqrqx/T2xO5VqOXxONeRfI=
github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4=
@ -1018,8 +1028,9 @@ github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPR
github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0=
go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
go.etcd.io/bbolt v1.3.5 h1:XAzx9gjCb0Rxj7EoqcClPD1d5ZBxZJk0jbuoPHenBt0=
go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ=
go.etcd.io/bbolt v1.3.6 h1:/ecaJf0sk1l4l6V4awd65v2C3ILy7MSj+s/x1ADCIMU=
go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4=
go.etcd.io/etcd v0.5.0-alpha.5.0.20190917205325-a14579fbfb1a/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg=
go.mongodb.org/mongo-driver v1.7.3/go.mod h1:NqaYOwnXWr5Pm7AOpO5QFxKJ503nbMse/R79oO62zWg=
go.mongodb.org/mongo-driver v1.7.5/go.mod h1:VXEWRZ6URJIkUq2SCAyapmhH0ZLRBP+FT4xhp5Zvxng=
@ -1083,6 +1094,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0
golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4=
golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM=
golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU=
golang.org/x/exp v0.0.0-20220827204233-334a2380cb91 h1:tnebWN09GYg9OLPss1KXj8txwZc6X6uMr6VFdcGNbHw=
golang.org/x/exp v0.0.0-20220827204233-334a2380cb91/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE=
golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js=
golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
@ -1263,6 +1276,7 @@ golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
@ -1315,13 +1329,6 @@ golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ=
golang.org/x/text v0.5.0 h1:OLmvp0KP+FVG99Ct/qFiL/Fhk4zp4QQnZ7b2U+5piUM=
golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8=
golang.org/x/time v0.0.0-20161028155119-f51c12702a4d/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20190921001708-c4c64cad1fd0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20200416051211-89c76fbcd5d1/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e h1:EHBhcS0mlXEAVwNyO2dLfjToGsyY4j24pTs2ScHnX7s=
golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=