Initial commit

Initial public review release v0.10.0
This commit is contained in:
alexvanin 2020-07-10 17:17:51 +03:00 committed by Stanislav Bogatyrev
commit dadfd90dcd
276 changed files with 46331 additions and 0 deletions

178
lib/placement/graph.go Normal file
View file

@ -0,0 +1,178 @@
package placement
import (
"github.com/gogo/protobuf/proto"
"github.com/multiformats/go-multiaddr"
"github.com/nspcc-dev/neofs-api-go/bootstrap"
"github.com/nspcc-dev/neofs-node/lib/netmap"
"github.com/pkg/errors"
)
// method returns copy of current Graph.
func (g *graph) copy() *graph {
var (
place *netmap.PlacementRule
roots = make([]*netmap.Bucket, 0, len(g.roots))
items = make([]bootstrap.NodeInfo, len(g.items))
)
copy(items, g.items)
for _, root := range g.roots {
var r *netmap.Bucket
if root != nil {
tmp := root.Copy()
r = &tmp
}
roots = append(roots, r)
}
place = proto.Clone(g.place).(*netmap.PlacementRule)
return &graph{
roots: roots,
items: items,
place: place,
}
}
func (g *graph) Exclude(list []multiaddr.Multiaddr) Graph {
if len(list) == 0 {
return g
}
var (
sub = g.copy()
ignore = make([]uint32, 0, len(list))
)
for i := range list {
for j := range sub.items {
if list[i].String() == sub.items[j].Address {
ignore = append(ignore, uint32(j))
}
}
}
return sub.Filter(func(group netmap.SFGroup, bucket *netmap.Bucket) *netmap.Bucket {
group.Exclude = ignore
return bucket.GetMaxSelection(group)
})
}
// Filter container by rules.
func (g *graph) Filter(rule FilterRule) Graph {
if rule == nil {
return g
}
var (
sub = g.copy()
roots = make([]*netmap.Bucket, len(g.roots))
items = make([]bootstrap.NodeInfo, len(g.items))
)
for i := range g.place.SFGroups {
if g.roots[i] == nil {
continue
}
root := g.roots[i].Copy()
roots[i] = rule(g.place.SFGroups[i], &root)
}
copy(items, g.items)
return &graph{
roots: roots,
items: items,
place: sub.place,
}
}
// NodeList returns slice of MultiAddresses for current graph.
func (g *graph) NodeList() ([]multiaddr.Multiaddr, error) {
var (
ln = uint32(len(g.items))
result = make([]multiaddr.Multiaddr, 0, ln)
items = make([]bootstrap.NodeInfo, len(g.items))
)
if ln == 0 {
return nil, ErrEmptyNodes
}
copy(items, g.items)
for _, root := range g.roots {
if root == nil {
continue
}
list := root.Nodelist()
if len(list) == 0 {
continue
}
for _, idx := range list {
if ln <= idx.N {
return nil, errors.Errorf("could not find index(%d) in list(size: %d)", ln, idx)
}
addr, err := multiaddr.NewMultiaddr(items[idx.N].Address)
if err != nil {
return nil, errors.Wrapf(err, "could not convert multi address(%s)", g.items[idx.N].Address)
}
result = append(result, addr)
}
}
if len(result) == 0 {
return nil, ErrEmptyNodes
}
return result, nil
}
// NodeInfo returns slice of NodeInfo for current graph.
func (g *graph) NodeInfo() ([]bootstrap.NodeInfo, error) {
var (
ln = uint32(len(g.items))
result = make([]bootstrap.NodeInfo, 0, ln)
items = make([]bootstrap.NodeInfo, len(g.items))
)
if ln == 0 {
return nil, ErrEmptyNodes
}
copy(items, g.items)
for _, root := range g.roots {
if root == nil {
continue
}
list := root.Nodelist()
if len(list) == 0 {
continue
}
for _, idx := range list {
if ln <= idx.N {
return nil, errors.Errorf("could not find index(%d) in list(size: %d)", ln, idx)
}
result = append(result, items[idx.N])
}
}
if len(result) == 0 {
return nil, ErrEmptyNodes
}
return result, nil
}

113
lib/placement/interface.go Normal file
View file

@ -0,0 +1,113 @@
package placement
import (
"context"
"github.com/multiformats/go-multiaddr"
"github.com/nspcc-dev/neofs-api-go/bootstrap"
"github.com/nspcc-dev/neofs-api-go/refs"
"github.com/nspcc-dev/neofs-node/lib/container"
"github.com/nspcc-dev/neofs-node/lib/netmap"
"github.com/nspcc-dev/neofs-node/lib/peers"
"go.uber.org/atomic"
"go.uber.org/zap"
)
type (
// Component is interface of placement service
Component interface {
// TODO leave for feature request
NetworkState() *bootstrap.SpreadMap
Neighbours(seed, epoch uint64, full bool) []peers.ID
Update(epoch uint64, nm *netmap.NetMap) error
Query(ctx context.Context, opts ...QueryOption) (Graph, error)
}
// QueryOptions for query request
QueryOptions struct {
CID refs.CID
Previous int
Excludes []multiaddr.Multiaddr
}
// QueryOption settings closure
QueryOption func(*QueryOptions)
// FilterRule bucket callback handler
FilterRule func(netmap.SFGroup, *netmap.Bucket) *netmap.Bucket
// Graph is result of request to Placement-component
Graph interface {
Filter(rule FilterRule) Graph
Exclude(list []multiaddr.Multiaddr) Graph
NodeList() ([]multiaddr.Multiaddr, error)
NodeInfo() ([]bootstrap.NodeInfo, error)
}
// Key to fetch node-list
Key []byte
// Params to create Placement component
Params struct {
Log *zap.Logger
Netmap *netmap.NetMap
Peerstore peers.Store
Fetcher container.Storage
ChronologyDuration uint64 // storing number of past epochs states
}
networkState struct {
nm *netmap.NetMap
epoch uint64
}
// placement is implementation of placement.Component
placement struct {
log *zap.Logger
cnr container.Storage
chronologyDur uint64
nmStore *netMapStore
ps peers.Store
healthy *atomic.Bool
}
// graph is implementation of placement.Graph
graph struct {
roots []*netmap.Bucket
items []bootstrap.NodeInfo
place *netmap.PlacementRule
}
)
// Copy network state.
func (ns networkState) Copy() *networkState {
return &networkState{
nm: ns.nm.Copy(),
epoch: ns.epoch,
}
}
// ExcludeNodes to ignore some nodes.
func ExcludeNodes(list []multiaddr.Multiaddr) QueryOption {
return func(opt *QueryOptions) {
opt.Excludes = list
}
}
// ContainerID set by Key.
func ContainerID(cid refs.CID) QueryOption {
return func(opt *QueryOptions) {
opt.CID = cid
}
}
// UsePreviousNetmap for query.
func UsePreviousNetmap(diff int) QueryOption {
return func(opt *QueryOptions) {
opt.Previous = diff
}
}

View file

@ -0,0 +1,69 @@
package placement
import (
"math"
"github.com/nspcc-dev/hrw"
"github.com/nspcc-dev/neofs-node/lib/netmap"
"github.com/nspcc-dev/neofs-node/lib/peers"
"go.uber.org/zap"
)
func calculateCount(n int) int {
if n < 30 {
return n
}
return int(1.4*math.Log(float64(n))+9) + 1
}
// Neighbours peers that which are distributed by hrw(seed)
// If full flag is set, all set of peers returns.
// Otherwise, result size depends on calculateCount function.
func (p *placement) Neighbours(seed, epoch uint64, full bool) []peers.ID {
nm := p.nmStore.get(epoch)
if nm == nil {
p.log.Error("could not receive network state",
zap.Uint64("epoch", epoch),
)
return nil
}
rPeers := p.listPeers(nm.ItemsCopy(), !full)
hrw.SortSliceByValue(rPeers, seed)
if full {
return rPeers
}
var (
ln = len(rPeers)
cut = calculateCount(ln)
)
if cut > ln {
cut = ln
}
return rPeers[:cut]
}
func (p *placement) listPeers(nodes netmap.Nodes, exclSelf bool) []peers.ID {
var (
id = p.ps.SelfID()
result = make([]peers.ID, 0, len(nodes))
)
for i := range nodes {
key := peers.IDFromBinary(nodes[i].PubKey)
if exclSelf && id.Equal(key) {
continue
}
result = append(result, key)
}
return result
}

View file

@ -0,0 +1,177 @@
package placement
import (
"crypto/ecdsa"
"strconv"
"testing"
"bou.ke/monkey"
"github.com/multiformats/go-multiaddr"
"github.com/nspcc-dev/neofs-api-go/bootstrap"
crypto "github.com/nspcc-dev/neofs-crypto"
"github.com/nspcc-dev/neofs-node/lib/netmap"
"github.com/nspcc-dev/neofs-node/lib/peers"
"github.com/nspcc-dev/neofs-node/lib/test"
"github.com/stretchr/testify/require"
)
func testAddress(t *testing.T) multiaddr.Multiaddr {
addr, err := multiaddr.NewMultiaddr("/ip4/0.0.0.0/tcp/0")
require.NoError(t, err)
return addr
}
// -- -- //
func testPeerstore(t *testing.T) peers.Store {
p, err := peers.NewStore(peers.StoreParams{
Key: test.DecodeKey(-1),
Logger: test.NewTestLogger(false),
Addr: testAddress(t),
})
require.NoError(t, err)
return p
}
const address = "/ip4/0.0.0.0/tcp/0/p2p/"
func TestPlacement_Neighbours(t *testing.T) {
t.Run("Placement component NPE fix test", func(t *testing.T) {
nodes := []bootstrap.NodeInfo{
{Address: address + idFromString(t, "USA1"), Options: []string{"/Location:Europe/Country:USA/City:NewYork"}},
{Address: address + idFromString(t, "ITL1"), Options: []string{"/Location:Europe/Country:Italy/City:Rome"}},
{Address: address + idFromString(t, "RUS1"), Options: []string{"/Location:Europe/Country:Russia/City:SPB"}},
}
ps := testPeerstore(t)
nm := testNetmap(t, nodes)
p := New(Params{
Log: test.NewTestLogger(false),
Peerstore: ps,
})
require.NotPanics(t, func() {
require.NoError(t, p.Update(1, nm))
})
})
t.Run("Placement Neighbours TestSuite", func(t *testing.T) {
keys := []*ecdsa.PrivateKey{
test.DecodeKey(0),
test.DecodeKey(1),
test.DecodeKey(2),
}
nodes := []bootstrap.NodeInfo{
{
Address: address + idFromString(t, "USA1"),
PubKey: crypto.MarshalPublicKey(&keys[0].PublicKey),
Options: []string{"/Location:Europe/Country:USA/City:NewYork"},
},
{
Address: address + idFromString(t, "ITL1"),
PubKey: crypto.MarshalPublicKey(&keys[1].PublicKey),
Options: []string{"/Location:Europe/Country:Italy/City:Rome"},
},
{
Address: address + idFromString(t, "RUS1"),
PubKey: crypto.MarshalPublicKey(&keys[2].PublicKey),
Options: []string{"/Location:Europe/Country:Russia/City:SPB"},
},
}
ps := testPeerstore(t)
nm := testNetmap(t, nodes)
p := New(Params{
Log: test.NewTestLogger(false),
Netmap: nm,
Peerstore: ps,
})
t.Run("check, that items have expected length (< 30)", func(t *testing.T) {
items := p.Neighbours(1, 0, false)
require.Len(t, items, len(nm.ItemsCopy()))
})
t.Run("check, that items have expected length ( > 30)", func(t *testing.T) {
opts := []string{"/Location:Europe/Country:Russia/City:SPB"}
key, err := ps.GetPublicKey(ps.SelfID())
require.NoError(t, err)
keyBytes := crypto.MarshalPublicKey(key)
addr := address + idFromString(t, "NewRUS")
err = nm.Add(addr, keyBytes, 0, opts...)
require.NoError(t, err)
for i := 0; i < 30; i++ {
addr := address + idFromString(t, "RUS"+strconv.Itoa(i+2))
key := test.DecodeKey(i + len(nodes))
pub := crypto.MarshalPublicKey(&key.PublicKey)
err := nm.Add(addr, pub, 0, opts...)
require.NoError(t, err)
}
ln := calculateCount(len(nm.ItemsCopy()))
items := p.Neighbours(1, 0, false)
require.Len(t, items, ln)
})
t.Run("check, that items is shuffled", func(t *testing.T) {
var cur, pre []peers.ID
for i := uint64(0); i < 10; i++ {
cur = p.Neighbours(i, 0, false)
require.NotEqual(t, pre, cur)
pre = cur
}
})
t.Run("check, that we can request more items that we have", func(t *testing.T) {
require.NotPanics(t, func() {
monkey.Patch(calculateCount, func(i int) int { return i + 1 })
defer monkey.Unpatch(calculateCount)
p.Neighbours(1, 0, false)
})
})
})
t.Run("unknown epoch", func(t *testing.T) {
s := &placement{
log: test.NewTestLogger(false),
nmStore: newNetMapStore(),
ps: testPeerstore(t),
}
require.Empty(t, s.Neighbours(1, 1, false))
})
t.Run("neighbors w/ set full flag", func(t *testing.T) {
var (
n = 3
e uint64 = 5
nm = netmap.NewNetmap()
nms = newNetMapStore()
)
for i := 0; i < n; i++ {
require.NoError(t, nm.Add("node"+strconv.Itoa(i), []byte{1}, 1))
}
nms.put(e, nm)
s := &placement{
log: test.NewTestLogger(false),
nmStore: nms,
ps: testPeerstore(t),
}
neighbors := s.Neighbours(1, e, true)
require.Len(t, neighbors, n)
})
}

257
lib/placement/placement.go Normal file
View file

@ -0,0 +1,257 @@
package placement
import (
"bytes"
"context"
"strings"
"github.com/nspcc-dev/neofs-api-go/bootstrap"
"github.com/nspcc-dev/neofs-api-go/refs"
crypto "github.com/nspcc-dev/neofs-crypto"
"github.com/nspcc-dev/neofs-node/internal"
"github.com/nspcc-dev/neofs-node/lib/container"
"github.com/nspcc-dev/neofs-node/lib/netmap"
"github.com/nspcc-dev/neofs-node/lib/peers"
"github.com/pkg/errors"
"go.uber.org/atomic"
"go.uber.org/zap"
)
const defaultChronologyDuration = 1
var (
// ErrEmptyNodes when container doesn't contains any nodes
ErrEmptyNodes = internal.Error("container doesn't contains nodes")
// ErrNodesBucketOmitted when in PlacementRule, Selector has not NodesBucket
ErrNodesBucketOmitted = internal.Error("nodes-bucket is omitted")
// ErrEmptyContainer when GetMaxSelection or GetSelection returns empty result
ErrEmptyContainer = internal.Error("could not get container, it's empty")
)
var errNilNetMap = errors.New("network map is nil")
// New is a placement component constructor.
func New(p Params) Component {
if p.Netmap == nil {
p.Netmap = netmap.NewNetmap()
}
if p.ChronologyDuration <= 0 {
p.ChronologyDuration = defaultChronologyDuration
}
pl := &placement{
log: p.Log,
cnr: p.Fetcher,
chronologyDur: p.ChronologyDuration,
nmStore: newNetMapStore(),
ps: p.Peerstore,
healthy: atomic.NewBool(false),
}
pl.nmStore.put(0, p.Netmap)
return pl
}
func (p *placement) Name() string { return "PresentInNetwork" }
func (p *placement) Healthy() bool { return p.healthy.Load() }
type strNodes []bootstrap.NodeInfo
func (n strNodes) String() string {
list := make([]string, 0, len(n))
for i := range n {
list = append(list, n[i].Address)
}
return `[` + strings.Join(list, ",") + `]`
}
func (p *placement) Update(epoch uint64, nm *netmap.NetMap) error {
cnm := p.nmStore.get(p.nmStore.epoch())
if cnm == nil {
return errNilNetMap
}
cp := cnm.Copy()
cp.Update(nm)
items := nm.ItemsCopy()
p.log.Debug("update to new netmap",
zap.Stringer("nodes", strNodes(items)))
p.log.Debug("update peerstore")
if err := p.ps.Update(cp); err != nil {
return err
}
var (
pubkeyBinary []byte
healthy bool
)
// storage nodes must be presented in network map to be healthy
pubkey, err := p.ps.GetPublicKey(p.ps.SelfID())
if err != nil {
p.log.Error("can't get my own public key")
}
pubkeyBinary = crypto.MarshalPublicKey(pubkey)
for i := range items {
if bytes.Equal(pubkeyBinary, items[i].GetPubKey()) {
healthy = true
}
p.log.Debug("new peer for dht",
zap.Stringer("peer", peers.IDFromBinary(items[i].GetPubKey())),
zap.String("addr", items[i].GetAddress()))
}
// make copy to previous
p.log.Debug("update previous netmap")
if epoch > p.chronologyDur {
p.nmStore.trim(epoch - p.chronologyDur)
}
p.log.Debug("update current netmap")
p.nmStore.put(epoch, cp)
p.log.Debug("update current epoch")
p.healthy.Store(healthy)
return nil
}
// NetworkState returns copy of current NetworkMap.
func (p *placement) NetworkState() *bootstrap.SpreadMap {
ns := p.networkState(p.nmStore.epoch())
if ns == nil {
ns = &networkState{nm: netmap.NewNetmap()}
}
return &bootstrap.SpreadMap{
Epoch: ns.epoch,
NetMap: ns.nm.Items(),
}
}
func (p *placement) networkState(epoch uint64) *networkState {
nm := p.nmStore.get(epoch)
if nm == nil {
return nil
}
return &networkState{
nm: nm.Copy(),
epoch: epoch,
}
}
// Query returns graph based on container.
func (p *placement) Query(ctx context.Context, opts ...QueryOption) (Graph, error) {
var (
items []bootstrap.NodeInfo
query QueryOptions
ignore []uint32
)
for _, opt := range opts {
opt(&query)
}
epoch := p.nmStore.epoch()
if query.Previous > 0 {
epoch -= uint64(query.Previous)
}
state := p.networkState(epoch)
if state == nil {
return nil, errors.Errorf("could not get network state for epoch #%d", epoch)
}
items = state.nm.Items()
gp := container.GetParams{}
gp.SetContext(ctx)
gp.SetCID(query.CID)
getRes, err := p.cnr.GetContainer(gp)
if err != nil {
return nil, errors.Wrap(err, "could not fetch container")
}
for i := range query.Excludes {
for j := range items {
if query.Excludes[i].String() == items[j].Address {
ignore = append(ignore, uint32(j))
}
}
}
rule := getRes.Container().GetRules()
return ContainerGraph(state.nm, &rule, ignore, query.CID)
}
// ContainerGraph applies the placement rules to network map and returns container graph.
func ContainerGraph(nm *netmap.NetMap, rule *netmap.PlacementRule, ignore []uint32, cid refs.CID) (Graph, error) {
root := nm.Root()
roots := make([]*netmap.Bucket, 0, len(rule.SFGroups))
for i := range rule.SFGroups {
rule.SFGroups[i].Exclude = ignore
if ln := len(rule.SFGroups[i].Selectors); ln <= 0 ||
rule.SFGroups[i].Selectors[ln-1].Key != netmap.NodesBucket {
return nil, errors.Wrapf(ErrNodesBucketOmitted, "container (%s)", cid)
}
bigSelectors := make([]netmap.Select, len(rule.SFGroups[i].Selectors))
for j := range rule.SFGroups[i].Selectors {
bigSelectors[j] = netmap.Select{
Key: rule.SFGroups[i].Selectors[j].Key,
Count: rule.SFGroups[i].Selectors[j].Count,
}
if rule.ReplFactor > 1 && rule.SFGroups[i].Selectors[j].Key == netmap.NodesBucket {
bigSelectors[j].Count *= rule.ReplFactor
}
}
sf := netmap.SFGroup{
Selectors: bigSelectors,
Filters: rule.SFGroups[i].Filters,
Exclude: ignore,
}
if tree := root.Copy().GetMaxSelection(sf); tree != nil {
// fetch graph for replication factor seeded by ContainerID
if tree = tree.GetSelection(bigSelectors, cid[:]); tree == nil {
return nil, errors.Wrapf(ErrEmptyContainer, "for container(%s) with repl-factor(%d)",
cid, rule.ReplFactor)
}
roots = append(roots, tree)
continue
}
return nil, errors.Wrap(ErrEmptyContainer, "empty for bigSelector")
}
return &graph{
roots: roots,
items: nm.ItemsCopy(),
place: rule,
}, nil
}

View file

@ -0,0 +1,407 @@
package placement
import (
"context"
"sort"
"strconv"
"strings"
"sync"
"testing"
"time"
"github.com/mr-tron/base58"
"github.com/multiformats/go-multiaddr"
"github.com/multiformats/go-multihash"
"github.com/nspcc-dev/neofs-api-go/bootstrap"
"github.com/nspcc-dev/neofs-api-go/container"
"github.com/nspcc-dev/neofs-api-go/refs"
libcnr "github.com/nspcc-dev/neofs-node/lib/container"
"github.com/nspcc-dev/neofs-node/lib/netmap"
"github.com/nspcc-dev/neofs-node/lib/peers"
"github.com/nspcc-dev/neofs-node/lib/test"
"github.com/pkg/errors"
"github.com/stretchr/testify/require"
)
type (
fakeDHT struct {
}
fakeContainerStorage struct {
libcnr.Storage
*sync.RWMutex
items map[refs.CID]*container.Container
}
)
var (
testDHTCapacity = 100
)
// -- -- //
func testContainerStorage() *fakeContainerStorage {
return &fakeContainerStorage{
RWMutex: new(sync.RWMutex),
items: make(map[refs.CID]*container.Container, testDHTCapacity),
}
}
func (f *fakeContainerStorage) GetContainer(p libcnr.GetParams) (*libcnr.GetResult, error) {
f.RLock()
val, ok := f.items[p.CID()]
f.RUnlock()
if !ok {
return nil, errors.New("value for requested key not found in DHT")
}
res := new(libcnr.GetResult)
res.SetContainer(val)
return res, nil
}
func (f *fakeContainerStorage) Put(c *container.Container) error {
id, err := c.ID()
if err != nil {
return err
}
f.Lock()
f.items[id] = c
f.Unlock()
return nil
}
func (f *fakeDHT) UpdatePeers([]peers.ID) {
// do nothing
}
func (f *fakeDHT) GetValue(ctx context.Context, key string) ([]byte, error) {
panic("implement me")
}
func (f *fakeDHT) PutValue(ctx context.Context, key string, val []byte) error {
panic("implement me")
}
func (f *fakeDHT) Get(ctx context.Context, key string) ([]byte, error) {
panic("implement me")
}
func (f *fakeDHT) Put(ctx context.Context, key string, val []byte) error {
panic("implement me")
}
// -- -- //
func testNetmap(t *testing.T, nodes []bootstrap.NodeInfo) *netmap.NetMap {
nm := netmap.NewNetmap()
for i := range nodes {
err := nm.Add(nodes[i].Address, nil, 0, nodes[i].Options...)
require.NoError(t, err)
}
return nm
}
// -- -- //
func idFromString(t *testing.T, id string) string {
buf, err := multihash.Encode([]byte(id), multihash.ID)
require.NoError(t, err)
return (multihash.Multihash(buf)).B58String()
}
func idFromAddress(t *testing.T, addr multiaddr.Multiaddr) string {
id, err := addr.ValueForProtocol(multiaddr.P_P2P)
require.NoError(t, err)
buf, err := base58.Decode(id)
require.NoError(t, err)
hs, err := multihash.Decode(buf)
require.NoError(t, err)
return string(hs.Digest)
}
// -- -- //
func TestPlacement(t *testing.T) {
multiaddr.SwapToP2pMultiaddrs()
testAddress := "/ip4/0.0.0.0/tcp/0/p2p/"
key := test.DecodeKey(-1)
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
ids := map[string]struct{}{
"GRM1": {}, "GRM2": {}, "GRM3": {}, "GRM4": {},
"SPN1": {}, "SPN2": {}, "SPN3": {}, "SPN4": {},
}
nodes := []bootstrap.NodeInfo{
{Address: testAddress + idFromString(t, "USA1"), Options: []string{"/Location:Europe/Country:USA/City:NewYork"}},
{Address: testAddress + idFromString(t, "ITL1"), Options: []string{"/Location:Europe/Country:Italy/City:Rome"}},
{Address: testAddress + idFromString(t, "RUS1"), Options: []string{"/Location:Europe/Country:Russia/City:SPB"}},
}
for id := range ids {
var opts []string
switch {
case strings.Contains(id, "GRM"):
opts = append(opts, "/Location:Europe/Country:Germany/City:"+id)
case strings.Contains(id, "SPN"):
opts = append(opts, "/Location:Europe/Country:Spain/City:"+id)
}
for i := 0; i < 4; i++ {
id := id + strconv.Itoa(i)
nodes = append(nodes, bootstrap.NodeInfo{
Address: testAddress + idFromString(t, id),
Options: opts,
})
}
}
sort.Slice(nodes, func(i, j int) bool {
return strings.Compare(nodes[i].Address, nodes[j].Address) == -1
})
nm := testNetmap(t, nodes)
cnrStorage := testContainerStorage()
p := New(Params{
Log: test.NewTestLogger(false),
Netmap: netmap.NewNetmap(),
Peerstore: testPeerstore(t),
Fetcher: cnrStorage,
})
require.NoError(t, p.Update(1, nm))
oid, err := refs.NewObjectID()
require.NoError(t, err)
// filter over oid
filter := func(group netmap.SFGroup, bucket *netmap.Bucket) *netmap.Bucket {
return bucket.GetSelection(group.Selectors, oid[:])
}
owner, err := refs.NewOwnerID(&key.PublicKey)
require.NoError(t, err)
res1, err := container.New(100, owner, 0, netmap.PlacementRule{
ReplFactor: 2,
SFGroups: []netmap.SFGroup{
{
Selectors: []netmap.Select{
{Key: "Country", Count: 1},
{Key: "City", Count: 2},
{Key: netmap.NodesBucket, Count: 1},
},
Filters: []netmap.Filter{
{Key: "Country", F: netmap.FilterIn("Germany", "Spain")},
},
},
},
})
require.NoError(t, err)
err = cnrStorage.Put(res1)
require.NoError(t, err)
res2, err := container.New(100, owner, 0, netmap.PlacementRule{
ReplFactor: 2,
SFGroups: []netmap.SFGroup{
{
Selectors: []netmap.Select{
{Key: "Country", Count: 1},
{Key: netmap.NodesBucket, Count: 10},
},
Filters: []netmap.Filter{
{Key: "Country", F: netmap.FilterIn("Germany", "Spain")},
},
},
},
})
require.NoError(t, err)
err = cnrStorage.Put(res2)
require.NoError(t, err)
res3, err := container.New(100, owner, 0, netmap.PlacementRule{
ReplFactor: 2,
SFGroups: []netmap.SFGroup{
{
Selectors: []netmap.Select{
{Key: "Country", Count: 1},
},
Filters: []netmap.Filter{
{Key: "Country", F: netmap.FilterIn("Germany", "Spain")},
},
},
},
})
require.NoError(t, err)
err = cnrStorage.Put(res3)
require.NoError(t, err)
t.Run("Should fail on empty container", func(t *testing.T) {
id, err := res2.ID()
require.NoError(t, err)
_, err = p.Query(ctx, ContainerID(id))
require.EqualError(t, errors.Cause(err), ErrEmptyContainer.Error())
})
t.Run("Should fail on Nodes Bucket is omitted in container", func(t *testing.T) {
id, err := res3.ID()
require.NoError(t, err)
_, err = p.Query(ctx, ContainerID(id))
require.EqualError(t, errors.Cause(err), ErrNodesBucketOmitted.Error())
})
t.Run("Should fail on unknown container (dht error)", func(t *testing.T) {
_, err = p.Query(ctx, ContainerID(refs.CID{5}))
require.Error(t, err)
})
id1, err := res1.ID()
require.NoError(t, err)
g, err := p.Query(ctx, ContainerID(id1))
require.NoError(t, err)
t.Run("Should return error on empty items", func(t *testing.T) {
_, err = g.Filter(func(netmap.SFGroup, *netmap.Bucket) *netmap.Bucket {
return &netmap.Bucket{}
}).NodeList()
require.EqualError(t, err, ErrEmptyNodes.Error())
})
t.Run("Should ignore some nodes", func(t *testing.T) {
g1, err := p.Query(ctx, ContainerID(id1))
require.NoError(t, err)
expect, err := g1.
Filter(filter).
NodeList()
require.NoError(t, err)
g2, err := p.Query(ctx, ContainerID(id1))
require.NoError(t, err)
actual, err := g2.
Filter(filter).
NodeList()
require.NoError(t, err)
require.Equal(t, expect, actual)
g3, err := p.Query(ctx, ContainerID(id1))
require.NoError(t, err)
actual, err = g3.
Exclude(expect).
Filter(filter).
NodeList()
require.NoError(t, err)
for _, item := range expect {
require.NotContains(t, actual, item)
}
g4, err := p.Query(ctx,
ContainerID(id1),
ExcludeNodes(expect))
require.NoError(t, err)
actual, err = g4.
Filter(filter).
NodeList()
require.NoError(t, err)
for _, item := range expect {
require.NotContains(t, actual, item)
}
})
t.Run("Should return error on nil Buckets", func(t *testing.T) {
_, err = g.Filter(func(netmap.SFGroup, *netmap.Bucket) *netmap.Bucket {
return nil
}).NodeList()
require.EqualError(t, err, ErrEmptyNodes.Error())
})
t.Run("Should return error on empty NodeInfo's", func(t *testing.T) {
cp := g.Filter(func(netmap.SFGroup, *netmap.Bucket) *netmap.Bucket {
return nil
})
cp.(*graph).items = nil
_, err := cp.NodeList()
require.EqualError(t, err, ErrEmptyNodes.Error())
})
t.Run("Should return error on unknown items", func(t *testing.T) {
cp := g.Filter(func(_ netmap.SFGroup, b *netmap.Bucket) *netmap.Bucket {
return b
})
cp.(*graph).items = cp.(*graph).items[:5]
_, err := cp.NodeList()
require.Error(t, err)
})
t.Run("Should return error on bad items", func(t *testing.T) {
cp := g.Filter(func(_ netmap.SFGroup, b *netmap.Bucket) *netmap.Bucket {
return b
})
for i := range cp.(*graph).items {
cp.(*graph).items[i].Address = "BadAddress"
}
_, err := cp.NodeList()
require.EqualError(t, errors.Cause(err), "failed to parse multiaddr \"BadAddress\": must begin with /")
})
list, err := g.
Filter(filter).
// must return same graph on empty filter
Filter(nil).
NodeList()
require.NoError(t, err)
// 1 Country, 2 Cities, 1 Node = 2 Nodes
require.Len(t, list, 2)
for _, item := range list {
id := idFromAddress(t, item)
require.Contains(t, ids, id[:4]) // exclude our postfix (0-4)
}
}
func TestContainerGraph(t *testing.T) {
t.Run("selectors index out-of-range", func(t *testing.T) {
rule := new(netmap.PlacementRule)
rule.SFGroups = append(rule.SFGroups, netmap.SFGroup{})
require.NotPanics(t, func() {
_, _ = ContainerGraph(
netmap.NewNetmap(),
rule,
nil,
refs.CID{},
)
})
})
}

66
lib/placement/store.go Normal file
View file

@ -0,0 +1,66 @@
package placement
import (
"sync"
"github.com/nspcc-dev/neofs-node/lib/netmap"
)
type (
// NetMap is a type alias of
// NetMap from netmap package.
NetMap = netmap.NetMap
netMapStore struct {
*sync.RWMutex
items map[uint64]*NetMap
curEpoch uint64
}
)
func newNetMapStore() *netMapStore {
return &netMapStore{
RWMutex: new(sync.RWMutex),
items: make(map[uint64]*NetMap),
}
}
func (s *netMapStore) put(epoch uint64, nm *NetMap) {
s.Lock()
s.items[epoch] = nm
s.curEpoch = epoch
s.Unlock()
}
func (s *netMapStore) get(epoch uint64) *NetMap {
s.RLock()
nm := s.items[epoch]
s.RUnlock()
return nm
}
// trim cleans all network states elder than epoch.
func (s *netMapStore) trim(epoch uint64) {
s.Lock()
m := make(map[uint64]struct{}, len(s.items))
for e := range s.items {
if e < epoch {
m[e] = struct{}{}
}
}
for e := range m {
delete(s.items, e)
}
s.Unlock()
}
func (s *netMapStore) epoch() uint64 {
s.RLock()
defer s.RUnlock()
return s.curEpoch
}