V3-1987: Optimize audits stats persistence (#2632)

* Added batch update stats for recordAuditSuccessStatus
* Added batch update stats to recordAuditFailStatus
* added configurable batch size
* build individual update/delete statements so the statements can be batched into 1 call to the DB
* notified #config-changes channel and ran make update-satellite-config-lock
* updated tests to use batch update stats
This commit is contained in:
ethanadams 2019-07-31 13:21:06 -04:00 committed by GitHub
parent 26a2fbb719
commit c9b46f2fe2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 685 additions and 279 deletions

View File

@ -123,6 +123,7 @@ func (planet *Planet) newSatellites(count int) ([]*satellite.Peer, error) {
UptimeReputationWeight: 1,
UptimeReputationDQ: 0.6,
},
UpdateStatsBatchSize: 100,
},
Discovery: discovery.Config{
DiscoveryInterval: 1 * time.Second,

View File

@ -205,19 +205,24 @@ func createData(planet *testplanet.Planet, days int) []testData {
func dqNodes(ctx *testcontext.Context, planet *testplanet.Planet) (map[storj.NodeID]bool, error) {
dqed := make(map[storj.NodeID]bool)
var updateRequests []*overlay.UpdateRequest
for i, n := range planet.StorageNodes {
if i%2 == 0 {
continue
}
_, err := planet.Satellites[0].Overlay.Service.UpdateStats(ctx, &overlay.UpdateRequest{
updateRequests = append(updateRequests, &overlay.UpdateRequest{
NodeID: n.ID(),
IsUp: true,
AuditSuccess: false,
})
if err != nil {
return nil, err
}
dqed[n.ID()] = true
}
_, err := planet.Satellites[0].Overlay.Service.BatchUpdateStats(ctx, updateRequests)
if err != nil {
return nil, err
}
for _, request := range updateRequests {
dqed[request.NodeID] = true
}
return dqed, nil
}

View File

@ -141,7 +141,7 @@ func TestContainUpdateStats(t *testing.T) {
require.NoError(t, err)
// update node stats
_, err = cache.UpdateStats(ctx, &overlay.UpdateRequest{NodeID: info1.NodeID})
_, err = cache.BatchUpdateStats(ctx, []*overlay.UpdateRequest{{NodeID: info1.NodeID}}, 100)
require.NoError(t, err)
// check contained flag set to false

View File

@ -217,7 +217,7 @@ func TestDisqualifiedNodeRemainsDisqualified(t *testing.T) {
assert.True(t, isDisqualified(t, ctx, satellitePeer, disqualifiedNode.ID()))
_, err = satellitePeer.DB.OverlayCache().UpdateStats(ctx, &overlay.UpdateRequest{
_, err = satellitePeer.DB.OverlayCache().BatchUpdateStats(ctx, []*overlay.UpdateRequest{{
NodeID: disqualifiedNode.ID(),
IsUp: true,
AuditSuccess: true,
@ -227,7 +227,7 @@ func TestDisqualifiedNodeRemainsDisqualified(t *testing.T) {
UptimeLambda: 0, // forget about history
UptimeWeight: 1,
UptimeDQ: 0, // make sure new reputation scores are larger than the DQ thresholds
})
}}, 100)
require.NoError(t, err)
assert.True(t, isDisqualified(t, ctx, satellitePeer, disqualifiedNode.ID()))
@ -241,7 +241,7 @@ func isDisqualified(t *testing.T, ctx *testcontext.Context, satellite *satellite
return node.Disqualified != nil
}
func disqualifyNode(t *testing.T, ctx *testcontext.Context, satellite *satellite.Peer, nodeID storj.NodeID) {
_, err := satellite.DB.OverlayCache().UpdateStats(ctx, &overlay.UpdateRequest{
_, err := satellite.DB.OverlayCache().BatchUpdateStats(ctx, []*overlay.UpdateRequest{{
NodeID: nodeID,
IsUp: true,
AuditSuccess: false,
@ -251,7 +251,7 @@ func disqualifyNode(t *testing.T, ctx *testcontext.Context, satellite *satellite
UptimeLambda: 1,
UptimeWeight: 1,
UptimeDQ: 0.5,
})
}}, 100)
require.NoError(t, err)
assert.True(t, isDisqualified(t, ctx, satellite, nodeID))
}

View File

@ -118,21 +118,21 @@ func (reporter *Reporter) RecordAudits(ctx context.Context, req *Report) (_ *Rep
// recordAuditFailStatus updates nodeIDs in overlay with isup=true, auditsuccess=false
func (reporter *Reporter) recordAuditFailStatus(ctx context.Context, failedAuditNodeIDs storj.NodeIDList) (failed storj.NodeIDList, err error) {
defer mon.Task()(&ctx)(&err)
var errlist errs.Group
for _, nodeID := range failedAuditNodeIDs {
_, err := reporter.overlay.UpdateStats(ctx, &overlay.UpdateRequest{
updateRequests := make([]*overlay.UpdateRequest, len(failedAuditNodeIDs))
for i, nodeID := range failedAuditNodeIDs {
updateRequests[i] = &overlay.UpdateRequest{
NodeID: nodeID,
IsUp: true,
AuditSuccess: false,
})
if err != nil {
failed = append(failed, nodeID)
errlist.Add(err)
}
}
if len(failed) > 0 {
reporter.log.Debug("failed to record Failed Nodes ", zap.Strings("NodeIDs", failed.Strings()))
return failed, errs.Combine(Error.New("failed to record some audit fail statuses in overlay"), errlist.Err())
if len(updateRequests) > 0 {
failed, err = reporter.overlay.BatchUpdateStats(ctx, updateRequests)
if err != nil || len(failed) > 0 {
reporter.log.Debug("failed to record Failed Nodes ", zap.Strings("NodeIDs", failed.Strings()))
return failed, errs.Combine(Error.New("failed to record some audit fail statuses in overlay"), err)
}
}
return nil, nil
}
@ -152,27 +152,29 @@ func (reporter *Reporter) recordOfflineStatus(ctx context.Context, offlineNodeID
reporter.log.Debug("failed to record Offline Nodes ", zap.Strings("NodeIDs", failed.Strings()))
return failed, errs.Combine(Error.New("failed to record some audit offline statuses in overlay"), errlist.Err())
}
return nil, nil
}
// recordAuditSuccessStatus updates nodeIDs in overlay with isup=true, auditsuccess=true
func (reporter *Reporter) recordAuditSuccessStatus(ctx context.Context, successNodeIDs storj.NodeIDList) (failed storj.NodeIDList, err error) {
defer mon.Task()(&ctx)(&err)
var errlist errs.Group
for _, nodeID := range successNodeIDs {
_, err := reporter.overlay.UpdateStats(ctx, &overlay.UpdateRequest{
updateRequests := make([]*overlay.UpdateRequest, len(successNodeIDs))
for i, nodeID := range successNodeIDs {
updateRequests[i] = &overlay.UpdateRequest{
NodeID: nodeID,
IsUp: true,
AuditSuccess: true,
})
if err != nil {
failed = append(failed, nodeID)
errlist.Add(err)
}
}
if len(failed) > 0 {
reporter.log.Debug("failed to record Success Nodes ", zap.Strings("NodeIDs", failed.Strings()))
return failed, errs.Combine(Error.New("failed to record some audit success statuses in overlay"), errlist.Err())
if len(updateRequests) > 0 {
failed, err = reporter.overlay.BatchUpdateStats(ctx, updateRequests)
if err != nil || len(failed) > 0 {
reporter.log.Debug("failed to record Success Nodes ", zap.Strings("NodeIDs", failed.Strings()))
return failed, errs.Combine(Error.New("failed to record some audit success statuses in overlay"), err)
}
}
return nil, nil
}
@ -181,6 +183,8 @@ func (reporter *Reporter) recordAuditSuccessStatus(ctx context.Context, successN
func (reporter *Reporter) recordPendingAudits(ctx context.Context, pendingAudits []*PendingAudit) (failed []*PendingAudit, err error) {
defer mon.Task()(&ctx)(&err)
var errlist errs.Group
var updateRequests []*overlay.UpdateRequest
for _, pendingAudit := range pendingAudits {
if pendingAudit.ReverifyCount < reporter.maxReverifyCount {
err := reporter.containment.IncrementPending(ctx, pendingAudit)
@ -190,22 +194,38 @@ func (reporter *Reporter) recordPendingAudits(ctx context.Context, pendingAudits
}
} else {
// record failure -- max reverify count reached
_, err := reporter.overlay.UpdateStats(ctx, &overlay.UpdateRequest{
updateRequests = append(updateRequests, &overlay.UpdateRequest{
NodeID: pendingAudit.NodeID,
IsUp: true,
AuditSuccess: false,
})
if err != nil {
failed = append(failed, pendingAudit)
errlist.Add(err)
}
}
}
if len(failed) > 0 {
for _, v := range failed {
reporter.log.Debug("failed to record Pending Nodes ", zap.Stringer("NodeID", v.NodeID), zap.String("Path", v.Path))
if len(updateRequests) > 0 {
failedBatch, err := reporter.overlay.BatchUpdateStats(ctx, updateRequests)
if err != nil {
errlist.Add(err)
}
if len(failedBatch) > 0 {
pendingMap := make(map[storj.NodeID]*PendingAudit)
for _, pendingAudit := range pendingAudits {
pendingMap[pendingAudit.NodeID] = pendingAudit
}
for _, nodeID := range failedBatch {
pending, ok := pendingMap[nodeID]
if ok {
failed = append(failed, pending)
}
}
}
if len(failed) > 0 {
for _, v := range failed {
reporter.log.Debug("failed to record Pending Nodes ", zap.Stringer("NodeID", v.NodeID), zap.String("Path", v.Path))
}
return failed, errs.Combine(Error.New("failed to record some pending audits"), errlist.Err())
}
return failed, errs.Combine(Error.New("failed to record some pending audits"), errlist.Err())
}
return nil, nil
}

View File

@ -82,6 +82,21 @@ func BenchmarkOverlay(b *testing.B) {
}
})
b.Run("BatchUpdateStats", func(b *testing.B) {
var updateRequests []*overlay.UpdateRequest
for i := 0; i < b.N; i++ {
id := all[i%len(all)]
updateRequests = append(updateRequests, &overlay.UpdateRequest{
NodeID: id,
AuditSuccess: i&1 == 0,
IsUp: i&2 == 0,
})
}
_, err := overlaydb.BatchUpdateStats(ctx, updateRequests, 100)
require.NoError(b, err)
})
b.Run("UpdateNodeInfo", func(b *testing.B) {
now := time.Now()
for i := 0; i < b.N; i++ {

View File

@ -61,6 +61,8 @@ type DB interface {
IsVetted(ctx context.Context, id storj.NodeID, criteria *NodeCriteria) (bool, error)
// Update updates node address
UpdateAddress(ctx context.Context, value *pb.Node, defaults NodeSelectionConfig) error
// BatchUpdateStats updates multiple storagenode's stats in one transaction
BatchUpdateStats(ctx context.Context, updateRequests []*UpdateRequest, batchSize int) (failed storj.NodeIDList, err error)
// UpdateStats all parts of single storagenode's stats.
UpdateStats(ctx context.Context, request *UpdateRequest) (stats *NodeStats, err error)
// UpdateNodeInfo updates node dossier with info requested from the node itself like node type, email, wallet, capacity, and version.
@ -138,17 +140,17 @@ type NodeStats struct {
// Cache is used to store and handle node information
type Cache struct {
log *zap.Logger
db DB
preferences NodeSelectionConfig
log *zap.Logger
db DB
config Config
}
// NewCache returns a new Cache
func NewCache(log *zap.Logger, db DB, preferences NodeSelectionConfig) *Cache {
func NewCache(log *zap.Logger, db DB, config Config) *Cache {
return &Cache{
log: log,
db: db,
preferences: preferences,
log: log,
db: db,
config: config,
}
}
@ -185,14 +187,14 @@ func (cache *Cache) Get(ctx context.Context, nodeID storj.NodeID) (_ *NodeDossie
// IsOnline checks if a node is 'online' based on the collected statistics.
func (cache *Cache) IsOnline(node *NodeDossier) bool {
return time.Now().Sub(node.Reputation.LastContactSuccess) < cache.preferences.OnlineWindow ||
return time.Now().Sub(node.Reputation.LastContactSuccess) < cache.config.Node.OnlineWindow ||
node.Reputation.LastContactSuccess.After(node.Reputation.LastContactFailure)
}
// FindStorageNodes searches the overlay network for nodes that meet the provided requirements
func (cache *Cache) FindStorageNodes(ctx context.Context, req FindStorageNodesRequest) (_ []*pb.Node, err error) {
defer mon.Task()(&ctx)(&err)
return cache.FindStorageNodesWithPreferences(ctx, req, &cache.preferences)
return cache.FindStorageNodesWithPreferences(ctx, req, &cache.config.Node)
}
// FindStorageNodesWithPreferences searches the overlay network for nodes that meet the provided criteria
@ -268,7 +270,7 @@ func (cache *Cache) FindStorageNodesWithPreferences(ctx context.Context, req Fin
func (cache *Cache) KnownOffline(ctx context.Context, nodeIds storj.NodeIDList) (offlineNodes storj.NodeIDList, err error) {
defer mon.Task()(&ctx)(&err)
criteria := &NodeCriteria{
OnlineWindow: cache.preferences.OnlineWindow,
OnlineWindow: cache.config.Node.OnlineWindow,
}
return cache.db.KnownOffline(ctx, criteria, nodeIds)
}
@ -277,7 +279,7 @@ func (cache *Cache) KnownOffline(ctx context.Context, nodeIds storj.NodeIDList)
func (cache *Cache) KnownUnreliableOrOffline(ctx context.Context, nodeIds storj.NodeIDList) (badNodes storj.NodeIDList, err error) {
defer mon.Task()(&ctx)(&err)
criteria := &NodeCriteria{
OnlineWindow: cache.preferences.OnlineWindow,
OnlineWindow: cache.config.Node.OnlineWindow,
}
return cache.db.KnownUnreliableOrOffline(ctx, criteria, nodeIds)
}
@ -286,7 +288,7 @@ func (cache *Cache) KnownUnreliableOrOffline(ctx context.Context, nodeIds storj.
func (cache *Cache) Reliable(ctx context.Context) (nodes storj.NodeIDList, err error) {
defer mon.Task()(&ctx)(&err)
criteria := &NodeCriteria{
OnlineWindow: cache.preferences.OnlineWindow,
OnlineWindow: cache.config.Node.OnlineWindow,
}
return cache.db.Reliable(ctx, criteria)
}
@ -311,15 +313,15 @@ func (cache *Cache) Put(ctx context.Context, nodeID storj.NodeID, value pb.Node)
if err != nil {
return OverlayError.Wrap(err)
}
return cache.db.UpdateAddress(ctx, &value, cache.preferences)
return cache.db.UpdateAddress(ctx, &value, cache.config.Node)
}
// IsVetted returns whether or not the node reaches reputable thresholds
func (cache *Cache) IsVetted(ctx context.Context, nodeID storj.NodeID) (reputable bool, err error) {
defer mon.Task()(&ctx)(&err)
criteria := &NodeCriteria{
AuditCount: cache.preferences.AuditCount,
UptimeCount: cache.preferences.UptimeCount,
AuditCount: cache.config.Node.AuditCount,
UptimeCount: cache.config.Node.UptimeCount,
}
reputable, err = cache.db.IsVetted(ctx, nodeID, criteria)
if err != nil {
@ -328,16 +330,31 @@ func (cache *Cache) IsVetted(ctx context.Context, nodeID storj.NodeID) (reputabl
return reputable, nil
}
// BatchUpdateStats updates multiple storagenode's stats in one transaction
func (cache *Cache) BatchUpdateStats(ctx context.Context, requests []*UpdateRequest) (failed storj.NodeIDList, err error) {
defer mon.Task()(&ctx)(&err)
for _, request := range requests {
request.AuditLambda = cache.config.Node.AuditReputationLambda
request.AuditWeight = cache.config.Node.AuditReputationWeight
request.AuditDQ = cache.config.Node.AuditReputationDQ
request.UptimeLambda = cache.config.Node.UptimeReputationLambda
request.UptimeWeight = cache.config.Node.UptimeReputationWeight
request.UptimeDQ = cache.config.Node.UptimeReputationDQ
}
return cache.db.BatchUpdateStats(ctx, requests, cache.config.UpdateStatsBatchSize)
}
// UpdateStats all parts of single storagenode's stats.
func (cache *Cache) UpdateStats(ctx context.Context, request *UpdateRequest) (stats *NodeStats, err error) {
defer mon.Task()(&ctx)(&err)
request.AuditLambda = cache.preferences.AuditReputationLambda
request.AuditWeight = cache.preferences.AuditReputationWeight
request.AuditDQ = cache.preferences.AuditReputationDQ
request.UptimeLambda = cache.preferences.UptimeReputationLambda
request.UptimeWeight = cache.preferences.UptimeReputationWeight
request.UptimeDQ = cache.preferences.UptimeReputationDQ
request.AuditLambda = cache.config.Node.AuditReputationLambda
request.AuditWeight = cache.config.Node.AuditReputationWeight
request.AuditDQ = cache.config.Node.AuditReputationDQ
request.UptimeLambda = cache.config.Node.UptimeReputationLambda
request.UptimeWeight = cache.config.Node.UptimeReputationWeight
request.UptimeDQ = cache.config.Node.UptimeReputationDQ
return cache.db.UpdateStats(ctx, request)
}
@ -351,9 +368,9 @@ func (cache *Cache) UpdateNodeInfo(ctx context.Context, node storj.NodeID, nodeI
// UpdateUptime updates a single storagenode's uptime stats.
func (cache *Cache) UpdateUptime(ctx context.Context, nodeID storj.NodeID, isUp bool) (stats *NodeStats, err error) {
defer mon.Task()(&ctx)(&err)
lambda := cache.preferences.UptimeReputationLambda
weight := cache.preferences.UptimeReputationWeight
uptimeDQ := cache.preferences.UptimeReputationDQ
lambda := cache.config.Node.UptimeReputationLambda
weight := cache.config.Node.UptimeReputationWeight
uptimeDQ := cache.config.Node.UptimeReputationDQ
return cache.db.UpdateUptime(ctx, nodeID, isUp, lambda, weight, uptimeDQ)
}
@ -363,9 +380,9 @@ func (cache *Cache) ConnFailure(ctx context.Context, node *pb.Node, failureError
var err error
defer mon.Task()(&ctx)(&err)
lambda := cache.preferences.UptimeReputationLambda
weight := cache.preferences.UptimeReputationWeight
uptimeDQ := cache.preferences.UptimeReputationDQ
lambda := cache.config.Node.UptimeReputationLambda
weight := cache.config.Node.UptimeReputationWeight
uptimeDQ := cache.config.Node.UptimeReputationDQ
// TODO: Kademlia paper specifies 5 unsuccessful PINGs before removing the node
// from our routing table, but this is the cache so maybe we want to treat
@ -386,9 +403,9 @@ func (cache *Cache) ConnSuccess(ctx context.Context, node *pb.Node) {
cache.log.Debug("error updating uptime for node", zap.Error(err))
}
lambda := cache.preferences.UptimeReputationLambda
weight := cache.preferences.UptimeReputationWeight
uptimeDQ := cache.preferences.UptimeReputationDQ
lambda := cache.config.Node.UptimeReputationLambda
weight := cache.config.Node.UptimeReputationWeight
uptimeDQ := cache.config.Node.UptimeReputationDQ
_, err = cache.db.UpdateUptime(ctx, node.Id, true, lambda, weight, uptimeDQ)
if err != nil {

View File

@ -68,7 +68,8 @@ func testCache(ctx context.Context, t *testing.T, store overlay.DB) {
address := &pb.NodeAddress{Address: "127.0.0.1:0"}
nodeSelectionConfig := testNodeSelectionConfig(0, 0, false)
cache := overlay.NewCache(zaptest.NewLogger(t), store, nodeSelectionConfig)
cacheConfig := overlay.Config{Node: nodeSelectionConfig, UpdateStatsBatchSize: 100}
cache := overlay.NewCache(zaptest.NewLogger(t), store, cacheConfig)
{ // Put
err := cache.Put(ctx, valid1ID, pb.Node{Id: valid1ID, Address: address})
@ -169,20 +170,21 @@ func testCache(ctx context.Context, t *testing.T, store overlay.DB) {
dqTime := *stats.Disqualified
// should not update once already disqualified
stats, err = cache.UpdateStats(ctx, &overlay.UpdateRequest{
_, err = cache.BatchUpdateStats(ctx, []*overlay.UpdateRequest{{
NodeID: valid2ID,
IsUp: false,
AuditSuccess: true,
})
}})
require.NoError(t, err)
dossier, err := cache.Get(ctx, valid2ID)
require.NoError(t, err)
require.EqualValues(t, stats.AuditReputationAlpha, nodeSelectionConfig.AuditReputationAlpha0)
require.EqualValues(t, stats.AuditReputationBeta, nodeSelectionConfig.AuditReputationBeta0)
require.EqualValues(t, stats.UptimeReputationAlpha, newUptimeAlpha)
require.EqualValues(t, stats.UptimeReputationBeta, newUptimeBeta)
require.NotNil(t, stats.Disqualified)
require.Equal(t, *stats.Disqualified, dqTime)
require.EqualValues(t, dossier.Reputation.AuditReputationAlpha, nodeSelectionConfig.AuditReputationAlpha0)
require.EqualValues(t, dossier.Reputation.AuditReputationBeta, nodeSelectionConfig.AuditReputationBeta0)
require.EqualValues(t, dossier.Reputation.UptimeReputationAlpha, newUptimeAlpha)
require.EqualValues(t, dossier.Reputation.UptimeReputationBeta, newUptimeBeta)
require.NotNil(t, dossier.Disqualified)
require.Equal(t, *dossier.Disqualified, dqTime)
}
}

View File

@ -19,7 +19,8 @@ var (
// Config is a configuration struct for everything you need to start the
// Overlay cache responsibility.
type Config struct {
Node NodeSelectionConfig
Node NodeSelectionConfig
UpdateStatsBatchSize int `help:"number of update requests to process per transaction" default:"100"`
}
// NodeSelectionConfig is a configuration struct to determine the minimum

View File

@ -54,7 +54,6 @@ func TestNodeSelection(t *testing.T) {
testplanet.Run(t, testplanet.Config{
SatelliteCount: 1, StorageNodeCount: 10, UplinkCount: 1,
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
var err error
satellite := planet.Satellites[0]
// This sets audit counts of 0, 1, 2, 3, ... 9
@ -72,99 +71,129 @@ func TestNodeSelection(t *testing.T) {
require.NoError(t, err)
}
}
testNodeSelection(t, ctx, planet)
})
}
// ensure all storagenodes are in overlay service
for _, storageNode := range planet.StorageNodes {
err = satellite.Overlay.Service.Put(ctx, storageNode.ID(), storageNode.Local().Node)
func TestNodeSelectionWithBatch(t *testing.T) {
testplanet.Run(t, testplanet.Config{
SatelliteCount: 1, StorageNodeCount: 10, UplinkCount: 1,
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
satellite := planet.Satellites[0]
// This sets audit counts of 0, 1, 2, 3, ... 9
// so that we can fine-tune how many nodes are considered new or reputable
// by modifying the audit count cutoff passed into FindStorageNodesWithPreferences
for i, node := range planet.StorageNodes {
for k := 0; k < i; k++ {
// These are done individually b/c the previous stat data is important
_, err := satellite.DB.OverlayCache().BatchUpdateStats(ctx, []*overlay.UpdateRequest{{
NodeID: node.ID(),
IsUp: true,
AuditSuccess: true,
AuditLambda: 1, AuditWeight: 1, AuditDQ: 0.5,
UptimeLambda: 1, UptimeWeight: 1, UptimeDQ: 0.5,
}}, 1)
require.NoError(t, err)
}
}
testNodeSelection(t, ctx, planet)
})
}
func testNodeSelection(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
satellite := planet.Satellites[0]
// ensure all storagenodes are in overlay service
for _, storageNode := range planet.StorageNodes {
err := satellite.Overlay.Service.Put(ctx, storageNode.ID(), storageNode.Local().Node)
assert.NoError(t, err)
}
type test struct {
Preferences overlay.NodeSelectionConfig
ExcludeCount int
RequestCount int
ExpectedCount int
ShouldFailWith *errs.Class
}
for i, tt := range []test{
{ // all reputable nodes, only reputable nodes requested
Preferences: testNodeSelectionConfig(0, 0, false),
RequestCount: 5,
ExpectedCount: 5,
},
{ // all reputable nodes, reputable and new nodes requested
Preferences: testNodeSelectionConfig(0, 1, false),
RequestCount: 5,
ExpectedCount: 5,
},
{ // 50-50 reputable and new nodes, not enough reputable nodes
Preferences: testNodeSelectionConfig(5, 0, false),
RequestCount: 10,
ExpectedCount: 5,
ShouldFailWith: &overlay.ErrNotEnoughNodes,
},
{ // 50-50 reputable and new nodes, reputable and new nodes requested, not enough reputable nodes
Preferences: testNodeSelectionConfig(5, 0.2, false),
RequestCount: 10,
ExpectedCount: 7,
ShouldFailWith: &overlay.ErrNotEnoughNodes,
},
{ // all new nodes except one, reputable and new nodes requested (happy path)
Preferences: testNodeSelectionConfig(9, 0.5, false),
RequestCount: 2,
ExpectedCount: 2,
},
{ // all new nodes except one, reputable and new nodes requested (not happy path)
Preferences: testNodeSelectionConfig(9, 0.5, false),
RequestCount: 4,
ExpectedCount: 3,
ShouldFailWith: &overlay.ErrNotEnoughNodes,
},
{ // all new nodes, reputable and new nodes requested
Preferences: testNodeSelectionConfig(50, 1, false),
RequestCount: 2,
ExpectedCount: 2,
},
{ // audit threshold edge case (1)
Preferences: testNodeSelectionConfig(9, 0, false),
RequestCount: 1,
ExpectedCount: 1,
},
{ // excluded node ids being excluded
Preferences: testNodeSelectionConfig(5, 0, false),
ExcludeCount: 7,
RequestCount: 5,
ExpectedCount: 3,
ShouldFailWith: &overlay.ErrNotEnoughNodes,
},
} {
t.Logf("#%2d. %+v", i, tt)
service := planet.Satellites[0].Overlay.Service
var excludedNodes []storj.NodeID
for _, storageNode := range planet.StorageNodes[:tt.ExcludeCount] {
excludedNodes = append(excludedNodes, storageNode.ID())
}
response, err := service.FindStorageNodesWithPreferences(ctx, overlay.FindStorageNodesRequest{
FreeBandwidth: 0,
FreeDisk: 0,
RequestedCount: tt.RequestCount,
ExcludedNodes: excludedNodes,
}, &tt.Preferences)
t.Log(len(response), err)
if tt.ShouldFailWith != nil {
assert.Error(t, err)
assert.True(t, tt.ShouldFailWith.Has(err))
} else {
assert.NoError(t, err)
}
type test struct {
Preferences overlay.NodeSelectionConfig
ExcludeCount int
RequestCount int
ExpectedCount int
ShouldFailWith *errs.Class
}
for i, tt := range []test{
{ // all reputable nodes, only reputable nodes requested
Preferences: testNodeSelectionConfig(0, 0, false),
RequestCount: 5,
ExpectedCount: 5,
},
{ // all reputable nodes, reputable and new nodes requested
Preferences: testNodeSelectionConfig(0, 1, false),
RequestCount: 5,
ExpectedCount: 5,
},
{ // 50-50 reputable and new nodes, not enough reputable nodes
Preferences: testNodeSelectionConfig(5, 0, false),
RequestCount: 10,
ExpectedCount: 5,
ShouldFailWith: &overlay.ErrNotEnoughNodes,
},
{ // 50-50 reputable and new nodes, reputable and new nodes requested, not enough reputable nodes
Preferences: testNodeSelectionConfig(5, 0.2, false),
RequestCount: 10,
ExpectedCount: 7,
ShouldFailWith: &overlay.ErrNotEnoughNodes,
},
{ // all new nodes except one, reputable and new nodes requested (happy path)
Preferences: testNodeSelectionConfig(9, 0.5, false),
RequestCount: 2,
ExpectedCount: 2,
},
{ // all new nodes except one, reputable and new nodes requested (not happy path)
Preferences: testNodeSelectionConfig(9, 0.5, false),
RequestCount: 4,
ExpectedCount: 3,
ShouldFailWith: &overlay.ErrNotEnoughNodes,
},
{ // all new nodes, reputable and new nodes requested
Preferences: testNodeSelectionConfig(50, 1, false),
RequestCount: 2,
ExpectedCount: 2,
},
{ // audit threshold edge case (1)
Preferences: testNodeSelectionConfig(9, 0, false),
RequestCount: 1,
ExpectedCount: 1,
},
{ // excluded node ids being excluded
Preferences: testNodeSelectionConfig(5, 0, false),
ExcludeCount: 7,
RequestCount: 5,
ExpectedCount: 3,
ShouldFailWith: &overlay.ErrNotEnoughNodes,
},
} {
t.Logf("#%2d. %+v", i, tt)
service := planet.Satellites[0].Overlay.Service
var excludedNodes []storj.NodeID
for _, storageNode := range planet.StorageNodes[:tt.ExcludeCount] {
excludedNodes = append(excludedNodes, storageNode.ID())
}
response, err := service.FindStorageNodesWithPreferences(ctx, overlay.FindStorageNodesRequest{
FreeBandwidth: 0,
FreeDisk: 0,
RequestedCount: tt.RequestCount,
ExcludedNodes: excludedNodes,
}, &tt.Preferences)
t.Log(len(response), err)
if tt.ShouldFailWith != nil {
assert.Error(t, err)
assert.True(t, tt.ShouldFailWith.Has(err))
} else {
assert.NoError(t, err)
}
assert.Equal(t, tt.ExpectedCount, len(response))
}
})
assert.Equal(t, tt.ExpectedCount, len(response))
}
}
func TestDistinctIPs(t *testing.T) {
@ -178,30 +207,6 @@ func TestDistinctIPs(t *testing.T) {
},
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
satellite := planet.Satellites[0]
service := satellite.Overlay.Service
tests := []struct {
nodeCount int
duplicateCount int
requestCount int
preferences overlay.NodeSelectionConfig
shouldFailWith *errs.Class
}{
{ // test only distinct IPs with half new nodes
requestCount: 4,
preferences: testNodeSelectionConfig(1, 0.5, true),
},
{ // test not enough distinct IPs
requestCount: 7,
preferences: testNodeSelectionConfig(0, 0, true),
shouldFailWith: &overlay.ErrNotEnoughNodes,
},
{ // test distinct flag false allows duplicates
duplicateCount: 10,
requestCount: 5,
preferences: testNodeSelectionConfig(0, 0.5, false),
},
}
// This sets a reputable audit count for nodes[8] and nodes[9].
for i := 9; i > 7; i-- {
_, err := satellite.DB.OverlayCache().UpdateStats(ctx, &overlay.UpdateRequest{
@ -217,35 +222,95 @@ func TestDistinctIPs(t *testing.T) {
})
assert.NoError(t, err)
}
for _, tt := range tests {
response, err := service.FindStorageNodesWithPreferences(ctx, overlay.FindStorageNodesRequest{
FreeBandwidth: 0,
FreeDisk: 0,
RequestedCount: tt.requestCount,
}, &tt.preferences)
if tt.shouldFailWith != nil {
assert.Error(t, err)
assert.True(t, tt.shouldFailWith.Has(err))
continue
} else {
require.NoError(t, err)
}
// assert all IPs are unique
if tt.preferences.DistinctIP {
ips := make(map[string]bool)
for _, n := range response {
assert.False(t, ips[n.LastIp])
ips[n.LastIp] = true
}
}
assert.Equal(t, tt.requestCount, len(response))
}
testDistinctIPs(t, ctx, planet)
})
}
func TestDistinctIPsWithBatch(t *testing.T) {
if runtime.GOOS == "darwin" {
t.Skip("Test does not work with macOS")
}
testplanet.Run(t, testplanet.Config{
SatelliteCount: 1, StorageNodeCount: 10, UplinkCount: 1,
Reconfigure: testplanet.Reconfigure{
NewIPCount: 3,
},
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
satellite := planet.Satellites[0]
// This sets a reputable audit count for nodes[8] and nodes[9].
for i := 9; i > 7; i-- {
// These are done individually b/c the previous stat data is important
_, err := satellite.DB.OverlayCache().BatchUpdateStats(ctx, []*overlay.UpdateRequest{{
NodeID: planet.StorageNodes[i].ID(),
IsUp: true,
AuditSuccess: true,
AuditLambda: 1,
AuditWeight: 1,
AuditDQ: 0.5,
UptimeLambda: 1,
UptimeWeight: 1,
UptimeDQ: 0.5,
}}, 1)
assert.NoError(t, err)
}
testDistinctIPs(t, ctx, planet)
})
}
func testDistinctIPs(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
satellite := planet.Satellites[0]
service := satellite.Overlay.Service
tests := []struct {
nodeCount int
duplicateCount int
requestCount int
preferences overlay.NodeSelectionConfig
shouldFailWith *errs.Class
}{
{ // test only distinct IPs with half new nodes
requestCount: 4,
preferences: testNodeSelectionConfig(1, 0.5, true),
},
{ // test not enough distinct IPs
requestCount: 7,
preferences: testNodeSelectionConfig(0, 0, true),
shouldFailWith: &overlay.ErrNotEnoughNodes,
},
{ // test distinct flag false allows duplicates
duplicateCount: 10,
requestCount: 5,
preferences: testNodeSelectionConfig(0, 0.5, false),
},
}
for _, tt := range tests {
response, err := service.FindStorageNodesWithPreferences(ctx, overlay.FindStorageNodesRequest{
FreeBandwidth: 0,
FreeDisk: 0,
RequestedCount: tt.requestCount,
}, &tt.preferences)
if tt.shouldFailWith != nil {
assert.Error(t, err)
assert.True(t, tt.shouldFailWith.Has(err))
continue
} else {
require.NoError(t, err)
}
// assert all IPs are unique
if tt.preferences.DistinctIP {
ips := make(map[string]bool)
for _, n := range response {
assert.False(t, ips[n.LastIp])
ips[n.LastIp] = true
}
}
assert.Equal(t, tt.requestCount, len(response))
}
}
func TestAddrtoNetwork_Conversion(t *testing.T) {
ctx := testcontext.New(t)

View File

@ -270,9 +270,8 @@ func New(log *zap.Logger, full *identity.FullIdentity, db DB, config *Config, ve
{ // setup overlay
log.Debug("Starting overlay")
config := config.Overlay
peer.Overlay.Service = overlay.NewCache(peer.Log.Named("overlay"), peer.DB.OverlayCache(), config.Node)
peer.Overlay.Service = overlay.NewCache(peer.Log.Named("overlay"), peer.DB.OverlayCache(), config.Overlay)
peer.Transport = peer.Transport.WithObservers(peer.Overlay.Service)
peer.Overlay.Inspector = overlay.NewInspector(peer.Overlay.Service)

View File

@ -21,7 +21,7 @@ func TestReliabilityCache_Concurrent(t *testing.T) {
ctx := testcontext.New(t)
defer ctx.Cleanup()
ocache := overlay.NewCache(zap.NewNop(), fakeOverlayDB{}, overlay.NodeSelectionConfig{})
ocache := overlay.NewCache(zap.NewNop(), fakeOverlayDB{}, overlay.Config{})
rcache := NewReliabilityCache(ocache, time.Millisecond)
for i := 0; i < 10; i++ {

View File

@ -941,6 +941,13 @@ func (m *lockedOverlayCache) UpdateNodeInfo(ctx context.Context, node storj.Node
return m.db.UpdateNodeInfo(ctx, node, nodeInfo)
}
// BatchUpdateStats updates multiple storagenode's stats in one transaction
func (m *lockedOverlayCache) BatchUpdateStats(ctx context.Context, request []*overlay.UpdateRequest, batchSize int) (failed storj.NodeIDList, err error) {
m.Lock()
defer m.Unlock()
return m.db.BatchUpdateStats(ctx, request, batchSize)
}
// UpdateStats all parts of single storagenode's stats.
func (m *lockedOverlayCache) UpdateStats(ctx context.Context, request *overlay.UpdateRequest) (stats *overlay.NodeStats, err error) {
m.Lock()

View File

@ -6,6 +6,8 @@ package satellitedb
import (
"context"
"database/sql"
"encoding/hex"
"fmt"
"strings"
"time"
@ -664,10 +666,83 @@ func (cache *overlaycache) UpdateAddress(ctx context.Context, info *pb.Node, def
return Error.Wrap(tx.Commit())
}
// BatchUpdateStats updates multiple storagenode's stats in one transaction
func (cache *overlaycache) BatchUpdateStats(ctx context.Context, updateRequests []*overlay.UpdateRequest, batchSize int) (failed storj.NodeIDList, err error) {
defer mon.Task()(&ctx)(&err)
if len(updateRequests) == 0 {
return failed, nil
}
doUpdate := func(updateSlice []*overlay.UpdateRequest) (duf storj.NodeIDList, err error) {
appendAll := func() {
for _, ur := range updateRequests {
duf = append(duf, ur.NodeID)
}
}
tx, err := cache.db.Open(ctx)
if err != nil {
appendAll()
return duf, Error.Wrap(err)
}
var allSQL string
for _, updateReq := range updateSlice {
dbNode, err := tx.Get_Node_By_Id(ctx, dbx.Node_Id(updateReq.NodeID.Bytes()))
if err != nil {
return nil, Error.Wrap(errs.Combine(err, tx.Rollback()))
}
// do not update reputation if node is disqualified
if dbNode.Disqualified != nil {
continue
}
updateNodeStats := populateUpdateNodeStats(dbNode, updateReq)
sql := buildUpdateStatement(cache.db, updateNodeStats)
allSQL += sql
}
if allSQL != "" {
results, err := tx.Tx.Exec(allSQL)
if results == nil || err != nil {
appendAll()
return duf, errs.Combine(err, tx.Rollback())
}
_, err = results.RowsAffected()
if err != nil {
appendAll()
return duf, errs.Combine(err, tx.Rollback())
}
}
return duf, Error.Wrap(tx.Commit())
}
var errlist errs.Group
length := len(updateRequests)
for i := 0; i < length; i += batchSize {
end := i + batchSize
if end > length {
end = length
}
failedBatch, err := doUpdate(updateRequests[i:end])
if err != nil && len(failedBatch) > 0 {
for _, fb := range failedBatch {
errlist.Add(err)
failed = append(failed, fb)
}
}
}
return failed, errlist.Err()
}
// UpdateStats a single storagenode's stats in the db
func (cache *overlaycache) UpdateStats(ctx context.Context, updateReq *overlay.UpdateRequest) (stats *overlay.NodeStats, err error) {
defer mon.Task()(&ctx)(&err)
nodeID := updateReq.NodeID
tx, err := cache.db.Open(ctx)
@ -683,62 +758,7 @@ func (cache *overlaycache) UpdateStats(ctx context.Context, updateReq *overlay.U
return getNodeStats(dbNode), Error.Wrap(tx.Commit())
}
auditAlpha, auditBeta, totalAuditCount := updateReputation(
updateReq.AuditSuccess,
dbNode.AuditReputationAlpha,
dbNode.AuditReputationBeta,
updateReq.AuditLambda,
updateReq.AuditWeight,
dbNode.TotalAuditCount,
)
mon.FloatVal("audit_reputation_alpha").Observe(auditAlpha)
mon.FloatVal("audit_reputation_beta").Observe(auditBeta)
uptimeAlpha, uptimeBeta, totalUptimeCount := updateReputation(
updateReq.IsUp,
dbNode.UptimeReputationAlpha,
dbNode.UptimeReputationBeta,
updateReq.UptimeLambda,
updateReq.UptimeWeight,
dbNode.TotalUptimeCount,
)
mon.FloatVal("uptime_reputation_alpha").Observe(uptimeAlpha)
mon.FloatVal("uptime_reputation_beta").Observe(uptimeBeta)
updateFields := dbx.Node_Update_Fields{
TotalAuditCount: dbx.Node_TotalAuditCount(totalAuditCount),
TotalUptimeCount: dbx.Node_TotalUptimeCount(totalUptimeCount),
AuditReputationAlpha: dbx.Node_AuditReputationAlpha(auditAlpha),
AuditReputationBeta: dbx.Node_AuditReputationBeta(auditBeta),
UptimeReputationAlpha: dbx.Node_UptimeReputationAlpha(uptimeAlpha),
UptimeReputationBeta: dbx.Node_UptimeReputationBeta(uptimeBeta),
}
auditRep := auditAlpha / (auditAlpha + auditBeta)
if auditRep <= updateReq.AuditDQ {
updateFields.Disqualified = dbx.Node_Disqualified(time.Now().UTC())
}
uptimeRep := uptimeAlpha / (uptimeAlpha + uptimeBeta)
if uptimeRep <= updateReq.UptimeDQ {
// n.b. that this will overwrite the audit DQ timestamp
// if it has already been set.
updateFields.Disqualified = dbx.Node_Disqualified(time.Now().UTC())
}
if updateReq.IsUp {
updateFields.UptimeSuccessCount = dbx.Node_UptimeSuccessCount(dbNode.UptimeSuccessCount + 1)
updateFields.LastContactSuccess = dbx.Node_LastContactSuccess(time.Now())
} else {
updateFields.LastContactFailure = dbx.Node_LastContactFailure(time.Now())
}
if updateReq.AuditSuccess {
updateFields.AuditSuccessCount = dbx.Node_AuditSuccessCount(dbNode.AuditSuccessCount + 1)
}
// Updating node stats always exits it from containment mode
updateFields.Contained = dbx.Node_Contained(false)
updateFields := populateUpdateFields(dbNode, updateReq)
dbNode, err = tx.Update_Node_By_Id(ctx, dbx.Node_Id(nodeID.Bytes()), updateFields)
if err != nil {
@ -979,3 +999,254 @@ func updateReputation(isSuccess bool, alpha, beta, lambda, w float64, totalCount
newBeta = lambda*beta + w*(1-v)/2
return newAlpha, newBeta, totalCount + 1
}
func buildUpdateStatement(db *dbx.DB, update updateNodeStats) string {
if update.NodeID.IsZero() {
return ""
}
atLeastOne := false
sql := "UPDATE nodes SET "
if update.TotalAuditCount.set {
atLeastOne = true
sql += fmt.Sprintf("total_audit_count = %v", update.TotalAuditCount.value)
}
if update.TotalUptimeCount.set {
if atLeastOne {
sql += ","
}
atLeastOne = true
sql += fmt.Sprintf("total_uptime_count = %v", update.TotalUptimeCount.value)
}
if update.AuditReputationAlpha.set {
if atLeastOne {
sql += ","
}
atLeastOne = true
sql += fmt.Sprintf("audit_reputation_alpha = %v", update.AuditReputationAlpha.value)
}
if update.AuditReputationBeta.set {
if atLeastOne {
sql += ","
}
atLeastOne = true
sql += fmt.Sprintf("audit_reputation_beta = %v", update.AuditReputationBeta.value)
}
if update.UptimeReputationAlpha.set {
if atLeastOne {
sql += ","
}
atLeastOne = true
sql += fmt.Sprintf("uptime_reputation_alpha = %v", update.UptimeReputationAlpha.value)
}
if update.UptimeReputationBeta.set {
if atLeastOne {
sql += ","
}
atLeastOne = true
sql += fmt.Sprintf("uptime_reputation_beta = %v", update.UptimeReputationBeta.value)
}
if update.Disqualified.set {
if atLeastOne {
sql += ","
}
atLeastOne = true
sql += fmt.Sprintf("disqualified = '%v'", update.Disqualified.value.Format(time.RFC3339Nano))
}
if update.UptimeSuccessCount.set {
if atLeastOne {
sql += ","
}
atLeastOne = true
sql += fmt.Sprintf("uptime_success_count = %v", update.UptimeSuccessCount.value)
}
if update.LastContactSuccess.set {
if atLeastOne {
sql += ","
}
atLeastOne = true
sql += fmt.Sprintf("last_contact_success = '%v'", update.LastContactSuccess.value.Format(time.RFC3339Nano))
}
if update.LastContactFailure.set {
if atLeastOne {
sql += ","
}
atLeastOne = true
sql += fmt.Sprintf("last_contact_failure = '%v'", update.LastContactFailure.value.Format(time.RFC3339Nano))
}
if update.AuditSuccessCount.set {
if atLeastOne {
sql += ","
}
atLeastOne = true
sql += fmt.Sprintf("audit_success_count = %v", update.AuditSuccessCount.value)
}
if update.Contained.set {
if atLeastOne {
sql += ","
}
atLeastOne = true
sql += fmt.Sprintf("contained = %v", update.Contained.value)
}
if !atLeastOne {
return ""
}
hexNodeID := hex.EncodeToString(update.NodeID.Bytes())
switch db.DB.Driver().(type) {
case *sqlite3.SQLiteDriver:
sql += fmt.Sprintf(" WHERE nodes.id = X'%v';\n", hexNodeID)
sql += fmt.Sprintf("DELETE FROM pending_audits WHERE pending_audits.node_id = X'%v';\n", hexNodeID)
case *pq.Driver:
sql += fmt.Sprintf(" WHERE nodes.id = decode('%v', 'hex');\n", hexNodeID)
sql += fmt.Sprintf("DELETE FROM pending_audits WHERE pending_audits.node_id = decode('%v', 'hex');\n", hexNodeID)
default:
return ""
}
return sql
}
type int64Field struct {
set bool
value int64
}
type float64Field struct {
set bool
value float64
}
type boolField struct {
set bool
value bool
}
type timeField struct {
set bool
value time.Time
}
type updateNodeStats struct {
NodeID storj.NodeID
TotalAuditCount int64Field
TotalUptimeCount int64Field
AuditReputationAlpha float64Field
AuditReputationBeta float64Field
UptimeReputationAlpha float64Field
UptimeReputationBeta float64Field
Disqualified timeField
UptimeSuccessCount int64Field
LastContactSuccess timeField
LastContactFailure timeField
AuditSuccessCount int64Field
Contained boolField
}
func populateUpdateNodeStats(dbNode *dbx.Node, updateReq *overlay.UpdateRequest) updateNodeStats {
auditAlpha, auditBeta, totalAuditCount := updateReputation(
updateReq.AuditSuccess,
dbNode.AuditReputationAlpha,
dbNode.AuditReputationBeta,
updateReq.AuditLambda,
updateReq.AuditWeight,
dbNode.TotalAuditCount,
)
mon.FloatVal("audit_reputation_alpha").Observe(auditAlpha)
mon.FloatVal("audit_reputation_beta").Observe(auditBeta)
uptimeAlpha, uptimeBeta, totalUptimeCount := updateReputation(
updateReq.IsUp,
dbNode.UptimeReputationAlpha,
dbNode.UptimeReputationBeta,
updateReq.UptimeLambda,
updateReq.UptimeWeight,
dbNode.TotalUptimeCount,
)
mon.FloatVal("uptime_reputation_alpha").Observe(uptimeAlpha)
mon.FloatVal("uptime_reputation_beta").Observe(uptimeBeta)
updateFields := updateNodeStats{
NodeID: updateReq.NodeID,
TotalAuditCount: int64Field{set: true, value: totalAuditCount},
TotalUptimeCount: int64Field{set: true, value: totalUptimeCount},
AuditReputationAlpha: float64Field{set: true, value: auditAlpha},
AuditReputationBeta: float64Field{set: true, value: auditBeta},
UptimeReputationAlpha: float64Field{set: true, value: uptimeAlpha},
UptimeReputationBeta: float64Field{set: true, value: uptimeBeta},
}
auditRep := auditAlpha / (auditAlpha + auditBeta)
if auditRep <= updateReq.AuditDQ {
updateFields.Disqualified = timeField{set: true, value: time.Now().UTC()}
}
uptimeRep := uptimeAlpha / (uptimeAlpha + uptimeBeta)
if uptimeRep <= updateReq.UptimeDQ {
// n.b. that this will overwrite the audit DQ timestamp
// if it has already been set.
updateFields.Disqualified = timeField{set: true, value: time.Now().UTC()}
}
if updateReq.IsUp {
updateFields.UptimeSuccessCount = int64Field{set: true, value: dbNode.UptimeSuccessCount + 1}
updateFields.LastContactSuccess = timeField{set: true, value: time.Now()}
} else {
updateFields.LastContactFailure = timeField{set: true, value: time.Now()}
}
if updateReq.AuditSuccess {
updateFields.AuditSuccessCount = int64Field{set: true, value: dbNode.AuditSuccessCount + 1}
}
// Updating node stats always exits it from containment mode
updateFields.Contained = boolField{set: true, value: false}
return updateFields
}
func populateUpdateFields(dbNode *dbx.Node, updateReq *overlay.UpdateRequest) dbx.Node_Update_Fields {
update := populateUpdateNodeStats(dbNode, updateReq)
updateFields := dbx.Node_Update_Fields{}
if update.TotalAuditCount.set {
updateFields.TotalAuditCount = dbx.Node_TotalAuditCount(update.TotalAuditCount.value)
}
if update.TotalUptimeCount.set {
updateFields.TotalUptimeCount = dbx.Node_TotalUptimeCount(update.TotalUptimeCount.value)
}
if update.AuditReputationAlpha.set {
updateFields.AuditReputationAlpha = dbx.Node_AuditReputationAlpha(update.AuditReputationAlpha.value)
}
if update.AuditReputationBeta.set {
updateFields.AuditReputationBeta = dbx.Node_AuditReputationBeta(update.AuditReputationBeta.value)
}
if update.UptimeReputationAlpha.set {
updateFields.UptimeReputationAlpha = dbx.Node_UptimeReputationAlpha(update.UptimeReputationAlpha.value)
}
if update.UptimeReputationBeta.set {
updateFields.UptimeReputationBeta = dbx.Node_UptimeReputationBeta(update.UptimeReputationBeta.value)
}
if update.Disqualified.set {
updateFields.Disqualified = dbx.Node_Disqualified(update.Disqualified.value)
}
if update.UptimeSuccessCount.set {
updateFields.UptimeSuccessCount = dbx.Node_UptimeSuccessCount(update.UptimeSuccessCount.value)
}
if update.LastContactSuccess.set {
updateFields.LastContactSuccess = dbx.Node_LastContactSuccess(update.LastContactSuccess.value)
}
if update.LastContactFailure.set {
updateFields.LastContactFailure = dbx.Node_LastContactFailure(update.LastContactFailure.value)
}
if update.AuditSuccessCount.set {
updateFields.AuditSuccessCount = dbx.Node_AuditSuccessCount(update.AuditSuccessCount.value)
}
if update.Contained.set {
updateFields.Contained = dbx.Node_Contained(update.Contained.value)
}
if updateReq.AuditSuccess {
updateFields.AuditSuccessCount = dbx.Node_AuditSuccessCount(dbNode.AuditSuccessCount + 1)
}
return updateFields
}

View File

@ -304,6 +304,9 @@ kademlia.operator.wallet: ""
# the normalization weight used to calculate the uptime SNs reputation
# overlay.node.uptime-reputation-weight: 1
# number of update requests to process per transaction
# overlay.update-stats-batch-size: 100
# how frequently repairer should try and repair more data
# repairer.interval: 1h0m0s