[#189] sdk/netmap: Get rid of the use of v2 types in the placement code

Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
This commit is contained in:
Leonard Lyubich 2020-11-05 15:45:06 +03:00 committed by Alex Vanin
parent 40d7c9cd93
commit 9834ed4e68
9 changed files with 195 additions and 225 deletions

View file

@ -4,7 +4,6 @@ import (
"errors"
"github.com/nspcc-dev/hrw"
"github.com/nspcc-dev/neofs-api-go/v2/netmap"
)
// Context contains references to named filters and cached numeric values.
@ -12,14 +11,14 @@ type Context struct {
// Netmap is a netmap structure to operate on.
Netmap *Netmap
// Filters stores processed filters.
Filters map[string]*netmap.Filter
Filters map[string]*Filter
// Selectors stores processed selectors.
Selectors map[string]*netmap.Selector
Selectors map[string]*Selector
// Selections stores result of selector processing.
Selections map[string][]Nodes
// numCache stores parsed numeric values.
numCache map[*netmap.Filter]uint64
numCache map[*Filter]uint64
// pivot is a seed for HRW.
pivot []byte
// pivotHash is a saved HRW hash of pivot
@ -50,11 +49,11 @@ var (
func NewContext(nm *Netmap) *Context {
return &Context{
Netmap: nm,
Filters: make(map[string]*netmap.Filter),
Selectors: make(map[string]*netmap.Selector),
Filters: make(map[string]*Filter),
Selectors: make(map[string]*Selector),
Selections: make(map[string][]Nodes),
numCache: make(map[*netmap.Filter]uint64),
numCache: make(map[*Filter]uint64),
aggregator: newMeanIQRAgg,
weightFunc: GetDefaultWeightFunc(nm.Nodes),
}

View file

@ -20,8 +20,8 @@ func (c *Context) applyFilter(name string, b *Node) bool {
}
// processFilters processes filters and returns error is any of them is invalid.
func (c *Context) processFilters(p *netmap.PlacementPolicy) error {
for _, f := range p.GetFilters() {
func (c *Context) processFilters(p *PlacementPolicy) error {
for _, f := range p.Filters() {
if err := c.processFilter(f, true); err != nil {
return err
}
@ -29,46 +29,46 @@ func (c *Context) processFilters(p *netmap.PlacementPolicy) error {
return nil
}
func (c *Context) processFilter(f *netmap.Filter, top bool) error {
func (c *Context) processFilter(f *Filter, top bool) error {
if f == nil {
return fmt.Errorf("%w: FILTER", ErrMissingField)
}
if f.GetName() == MainFilterName {
if f.Name() == MainFilterName {
return fmt.Errorf("%w: '*' is reserved", ErrInvalidFilterName)
}
if top && f.GetName() == "" {
if top && f.Name() == "" {
return ErrUnnamedTopFilter
}
if !top && f.GetName() != "" && c.Filters[f.GetName()] == nil {
return fmt.Errorf("%w: '%s'", ErrFilterNotFound, f.GetName())
if !top && f.Name() != "" && c.Filters[f.Name()] == nil {
return fmt.Errorf("%w: '%s'", ErrFilterNotFound, f.Name())
}
switch f.GetOp() {
case netmap.AND, netmap.OR:
for _, flt := range f.GetFilters() {
switch f.Operation() {
case OpAND, OpOR:
for _, flt := range f.InnerFilters() {
if err := c.processFilter(flt, false); err != nil {
return err
}
}
default:
if len(f.GetFilters()) != 0 {
if len(f.InnerFilters()) != 0 {
return ErrNonEmptyFilters
} else if !top && f.GetName() != "" { // named reference
} else if !top && f.Name() != "" { // named reference
return nil
}
switch f.GetOp() {
case netmap.EQ, netmap.NE:
case netmap.GT, netmap.GE, netmap.LT, netmap.LE:
n, err := strconv.ParseUint(f.GetValue(), 10, 64)
switch f.Operation() {
case OpEQ, OpNE:
case OpGT, OpGE, OpLT, OpLE:
n, err := strconv.ParseUint(f.Value(), 10, 64)
if err != nil {
return fmt.Errorf("%w: '%s'", ErrInvalidNumber, f.GetValue())
return fmt.Errorf("%w: '%s'", ErrInvalidNumber, f.Value())
}
c.numCache[f] = n
default:
return fmt.Errorf("%w: %d", ErrInvalidFilterOp, f.GetOp())
return fmt.Errorf("%w: %s", ErrInvalidFilterOp, f.Operation())
}
}
if top {
c.Filters[f.GetName()] = f
c.Filters[f.Name()] = f
}
return nil
}
@ -76,54 +76,54 @@ func (c *Context) processFilter(f *netmap.Filter, top bool) error {
// match matches f against b. It returns no errors because
// filter should have been parsed during context creation
// and missing node properties are considered as a regular fail.
func (c *Context) match(f *netmap.Filter, b *Node) bool {
switch f.GetOp() {
case netmap.AND, netmap.OR:
for _, lf := range f.GetFilters() {
if lf.GetName() != "" {
lf = c.Filters[lf.GetName()]
func (c *Context) match(f *Filter, b *Node) bool {
switch f.Operation() {
case OpAND, OpOR:
for _, lf := range f.InnerFilters() {
if lf.Name() != "" {
lf = c.Filters[lf.Name()]
}
ok := c.match(lf, b)
if ok == (f.GetOp() == netmap.OR) {
if ok == (f.Operation() == OpOR) {
return ok
}
}
return f.GetOp() == netmap.AND
return f.Operation() == OpAND
default:
return c.matchKeyValue(f, b)
}
}
func (c *Context) matchKeyValue(f *netmap.Filter, b *Node) bool {
switch f.GetOp() {
case netmap.EQ:
return b.Attribute(f.GetKey()) == f.GetValue()
case netmap.NE:
return b.Attribute(f.GetKey()) != f.GetValue()
func (c *Context) matchKeyValue(f *Filter, b *Node) bool {
switch f.Operation() {
case OpEQ:
return b.Attribute(f.Key()) == f.Value()
case OpNE:
return b.Attribute(f.Key()) != f.Value()
default:
var attr uint64
switch f.GetKey() {
switch f.Key() {
case PriceAttr:
attr = b.Price
case CapacityAttr:
attr = b.Capacity
default:
var err error
attr, err = strconv.ParseUint(b.Attribute(f.GetKey()), 10, 64)
attr, err = strconv.ParseUint(b.Attribute(f.Key()), 10, 64)
if err != nil {
// Note: because filters are somewhat independent from nodes attributes,
// We don't report an error here, and fail filter instead.
return false
}
}
switch f.GetOp() {
case netmap.GT:
switch f.Operation() {
case OpGT:
return attr > c.numCache[f]
case netmap.GE:
case OpGE:
return attr >= c.numCache[f]
case netmap.LT:
case OpLT:
return attr < c.numCache[f]
case netmap.LE:
case OpLE:
return attr <= c.numCache[f]
}
}

View file

@ -9,12 +9,12 @@ import (
)
func TestContext_ProcessFilters(t *testing.T) {
fs := []*netmap.Filter{
newFilter("StorageSSD", "Storage", "SSD", netmap.EQ),
newFilter("GoodRating", "Rating", "4", netmap.GE),
newFilter("Main", "", "", netmap.AND,
fs := []*Filter{
newFilter("StorageSSD", "Storage", "SSD", OpEQ),
newFilter("GoodRating", "Rating", "4", OpGE),
newFilter("Main", "", "", OpAND,
newFilter("StorageSSD", "", "", 0),
newFilter("", "IntField", "123", netmap.LT),
newFilter("", "IntField", "123", OpLT),
newFilter("GoodRating", "", "", 0)),
}
nm, err := NewNetmap(nil)
@ -24,49 +24,49 @@ func TestContext_ProcessFilters(t *testing.T) {
require.NoError(t, c.processFilters(p))
require.Equal(t, 3, len(c.Filters))
for _, f := range fs {
require.Equal(t, f, c.Filters[f.GetName()])
require.Equal(t, f, c.Filters[f.Name()])
}
require.Equal(t, uint64(4), c.numCache[fs[1]])
require.Equal(t, uint64(123), c.numCache[fs[2].GetFilters()[1]])
require.Equal(t, uint64(123), c.numCache[fs[2].InnerFilters()[1]])
}
func TestContext_ProcessFiltersInvalid(t *testing.T) {
errTestCases := []struct {
name string
filter *netmap.Filter
filter *Filter
err error
}{
{
"UnnamedTop",
newFilter("", "Storage", "SSD", netmap.EQ),
newFilter("", "Storage", "SSD", OpEQ),
ErrUnnamedTopFilter,
},
{
"InvalidReference",
newFilter("Main", "", "", netmap.AND,
newFilter("Main", "", "", OpAND,
newFilter("StorageSSD", "", "", 0)),
ErrFilterNotFound,
},
{
"NonEmptyKeyed",
newFilter("Main", "Storage", "SSD", netmap.EQ,
newFilter("Main", "Storage", "SSD", OpEQ,
newFilter("StorageSSD", "", "", 0)),
ErrNonEmptyFilters,
},
{
"InvalidNumber",
newFilter("Main", "Rating", "three", netmap.GE),
newFilter("Main", "Rating", "three", OpGE),
ErrInvalidNumber,
},
{
"InvalidOp",
newFilter("Main", "Rating", "3", netmap.UnspecifiedOperation),
newFilter("Main", "Rating", "3", 0),
ErrInvalidFilterOp,
},
{
"InvalidName",
newFilter("*", "Rating", "3", netmap.GE),
newFilter("*", "Rating", "3", OpGE),
ErrInvalidFilterName,
},
{
@ -78,7 +78,7 @@ func TestContext_ProcessFiltersInvalid(t *testing.T) {
for _, tc := range errTestCases {
t.Run(tc.name, func(t *testing.T) {
c := NewContext(new(Netmap))
p := newPlacementPolicy(1, nil, nil, []*netmap.Filter{tc.filter})
p := newPlacementPolicy(1, nil, nil, []*Filter{tc.filter})
err := c.processFilters(p)
require.True(t, errors.Is(err, tc.err), "got: %v", err)
})
@ -93,87 +93,87 @@ func TestFilter_MatchSimple(t *testing.T) {
testCases := []struct {
name string
ok bool
f *netmap.Filter
f *Filter
}{
{
"GE_true", true,
newFilter("Main", "Rating", "4", netmap.GE),
newFilter("Main", "Rating", "4", OpGE),
},
{
"GE_false", false,
newFilter("Main", "Rating", "5", netmap.GE),
newFilter("Main", "Rating", "5", OpGE),
},
{
"GT_true", true,
newFilter("Main", "Rating", "3", netmap.GT),
newFilter("Main", "Rating", "3", OpGT),
},
{
"GT_false", false,
newFilter("Main", "Rating", "4", netmap.GT),
newFilter("Main", "Rating", "4", OpGT),
},
{
"LE_true", true,
newFilter("Main", "Rating", "4", netmap.LE),
newFilter("Main", "Rating", "4", OpLE),
},
{
"LE_false", false,
newFilter("Main", "Rating", "3", netmap.LE),
newFilter("Main", "Rating", "3", OpLE),
},
{
"LT_true", true,
newFilter("Main", "Rating", "5", netmap.LT),
newFilter("Main", "Rating", "5", OpLT),
},
{
"LT_false", false,
newFilter("Main", "Rating", "4", netmap.LT),
newFilter("Main", "Rating", "4", OpLT),
},
{
"EQ_true", true,
newFilter("Main", "Country", "Germany", netmap.EQ),
newFilter("Main", "Country", "Germany", OpEQ),
},
{
"EQ_false", false,
newFilter("Main", "Country", "China", netmap.EQ),
newFilter("Main", "Country", "China", OpEQ),
},
{
"NE_true", true,
newFilter("Main", "Country", "France", netmap.NE),
newFilter("Main", "Country", "France", OpNE),
},
{
"NE_false", false,
newFilter("Main", "Country", "Germany", netmap.NE),
newFilter("Main", "Country", "Germany", OpNE),
},
}
for _, tc := range testCases {
c := NewContext(new(Netmap))
p := newPlacementPolicy(1, nil, nil, []*netmap.Filter{tc.f})
p := newPlacementPolicy(1, nil, nil, []*Filter{tc.f})
require.NoError(t, c.processFilters(p))
require.Equal(t, tc.ok, c.match(tc.f, b))
}
t.Run("InvalidOp", func(t *testing.T) {
f := newFilter("Main", "Rating", "5", netmap.EQ)
f := newFilter("Main", "Rating", "5", OpEQ)
c := NewContext(new(Netmap))
p := newPlacementPolicy(1, nil, nil, []*netmap.Filter{f})
p := newPlacementPolicy(1, nil, nil, []*Filter{f})
require.NoError(t, c.processFilters(p))
// just for the coverage
f.SetOp(netmap.UnspecifiedOperation)
f.SetOperation(0)
require.False(t, c.match(f, b))
})
}
func TestFilter_Match(t *testing.T) {
fs := []*netmap.Filter{
newFilter("StorageSSD", "Storage", "SSD", netmap.EQ),
newFilter("GoodRating", "Rating", "4", netmap.GE),
newFilter("Main", "", "", netmap.AND,
fs := []*Filter{
newFilter("StorageSSD", "Storage", "SSD", OpEQ),
newFilter("GoodRating", "Rating", "4", OpGE),
newFilter("Main", "", "", OpAND,
newFilter("StorageSSD", "", "", 0),
newFilter("", "IntField", "123", netmap.LT),
newFilter("", "IntField", "123", OpLT),
newFilter("GoodRating", "", "", 0),
newFilter("", "", "", netmap.OR,
newFilter("", "Param", "Value1", netmap.EQ),
newFilter("", "Param", "Value2", netmap.EQ),
newFilter("", "", "", OpOR,
newFilter("", "Param", "Value1", OpEQ),
newFilter("", "Param", "Value2", OpEQ),
)),
}
c := NewContext(new(Netmap))

View file

@ -1,21 +1,17 @@
package netmap
import (
"github.com/nspcc-dev/neofs-api-go/v2/netmap"
)
func newFilter(name string, k, v string, op netmap.Operation, fs ...*netmap.Filter) *netmap.Filter {
f := new(netmap.Filter)
func newFilter(name string, k, v string, op Operation, fs ...*Filter) *Filter {
f := NewFilter()
f.SetName(name)
f.SetKey(k)
f.SetOp(op)
f.SetOperation(op)
f.SetValue(v)
f.SetFilters(fs)
f.SetInnerFilters(fs...)
return f
}
func newSelector(name string, attr string, c netmap.Clause, count uint32, filter string) *netmap.Selector {
s := new(netmap.Selector)
func newSelector(name string, attr string, c Clause, count uint32, filter string) *Selector {
s := NewSelector()
s.SetName(name)
s.SetAttribute(attr)
s.SetCount(count)
@ -24,32 +20,32 @@ func newSelector(name string, attr string, c netmap.Clause, count uint32, filter
return s
}
func newPlacementPolicy(bf uint32, rs []*netmap.Replica, ss []*netmap.Selector, fs []*netmap.Filter) *netmap.PlacementPolicy {
p := new(netmap.PlacementPolicy)
func newPlacementPolicy(bf uint32, rs []*Replica, ss []*Selector, fs []*Filter) *PlacementPolicy {
p := NewPlacementPolicy()
p.SetContainerBackupFactor(bf)
p.SetReplicas(rs)
p.SetSelectors(ss)
p.SetFilters(fs)
p.SetReplicas(rs...)
p.SetSelectors(ss...)
p.SetFilters(fs...)
return p
}
func newReplica(c uint32, s string) *netmap.Replica {
r := new(netmap.Replica)
func newReplica(c uint32, s string) *Replica {
r := NewReplica()
r.SetCount(c)
r.SetSelector(s)
return r
}
func nodeInfoFromAttributes(props ...string) netmap.NodeInfo {
attrs := make([]*netmap.Attribute, len(props)/2)
func nodeInfoFromAttributes(props ...string) NodeInfo {
attrs := make([]*NodeAttribute, len(props)/2)
for i := range attrs {
attrs[i] = new(netmap.Attribute)
attrs[i] = NewNodeAttribute()
attrs[i].SetKey(props[i*2])
attrs[i].SetValue(props[i*2+1])
}
var n netmap.NodeInfo
n.SetAttributes(attrs)
return n
n := NewNodeInfo()
n.SetAttributes(attrs...)
return *n
}
func getTestNode(props ...string) *Node {

View file

@ -4,7 +4,6 @@ import (
"fmt"
"github.com/nspcc-dev/hrw"
"github.com/nspcc-dev/neofs-api-go/v2/netmap"
)
// Netmap represents netmap which contains preprocessed nodes.
@ -43,7 +42,7 @@ func (m *Netmap) GetPlacementVectors(cnt ContainerNodes, pivot []byte) ([]Nodes,
// GetContainerNodes returns nodes corresponding to each replica.
// Order of returned nodes corresponds to order of replicas in p.
// pivot is a seed for HRW sorting.
func (m *Netmap) GetContainerNodes(p *netmap.PlacementPolicy, pivot []byte) (ContainerNodes, error) {
func (m *Netmap) GetContainerNodes(p *PlacementPolicy, pivot []byte) (ContainerNodes, error) {
c := NewContext(m)
c.setPivot(pivot)
if err := c.processFilters(p); err != nil {
@ -52,19 +51,19 @@ func (m *Netmap) GetContainerNodes(p *netmap.PlacementPolicy, pivot []byte) (Con
if err := c.processSelectors(p); err != nil {
return nil, err
}
result := make([]Nodes, len(p.GetReplicas()))
for i, r := range p.GetReplicas() {
result := make([]Nodes, len(p.Replicas()))
for i, r := range p.Replicas() {
if r == nil {
return nil, fmt.Errorf("%w: REPLICA", ErrMissingField)
}
if r.GetSelector() == "" {
for _, s := range p.GetSelectors() {
result[i] = append(result[i], flattenNodes(c.Selections[s.GetName()])...)
if r.Selector() == "" {
for _, s := range p.Selectors() {
result[i] = append(result[i], flattenNodes(c.Selections[s.Name()])...)
}
}
nodes, ok := c.Selections[r.GetSelector()]
nodes, ok := c.Selections[r.Selector()]
if !ok {
return nil, fmt.Errorf("%w: REPLICA '%s'", ErrSelectorNotFound, r.GetSelector())
return nil, fmt.Errorf("%w: REPLICA '%s'", ErrSelectorNotFound, r.Selector())
}
result[i] = append(result[i], flattenNodes(nodes)...)

View file

@ -16,7 +16,7 @@ type (
Price uint64
AttrMap map[string]string
InfoV2 *netmap.NodeInfo
*NodeInfo
}
// Nodes represents slice of graph leafs.
@ -57,19 +57,8 @@ func (n Node) Hash() uint64 {
return n.ID
}
// NetworkAddress returns network address
// of the node in a string format.
func (n Node) NetworkAddress() string {
return n.InfoV2.GetAddress()
}
// PublicKey returns public key of the node in bytes.
func (n Node) PublicKey() []byte {
return n.InfoV2.GetPublicKey()
}
// NodesFromV2 converts slice of v2 netmap.NodeInfo to a generic node slice.
func NodesFromV2(infos []netmap.NodeInfo) Nodes {
// NodesFromInfo converts slice of NodeInfo to a generic node slice.
func NodesFromInfo(infos []NodeInfo) Nodes {
nodes := make(Nodes, len(infos))
for i := range infos {
nodes[i] = newNodeV2(i, &infos[i])
@ -77,21 +66,21 @@ func NodesFromV2(infos []netmap.NodeInfo) Nodes {
return nodes
}
func newNodeV2(index int, ni *netmap.NodeInfo) *Node {
func newNodeV2(index int, ni *NodeInfo) *Node {
n := &Node{
ID: hrw.Hash(ni.GetPublicKey()),
Index: index,
AttrMap: make(map[string]string, len(ni.GetAttributes())),
InfoV2: ni,
ID: hrw.Hash(ni.PublicKey()),
Index: index,
AttrMap: make(map[string]string, len(ni.Attributes())),
NodeInfo: ni,
}
for _, attr := range ni.GetAttributes() {
switch attr.GetKey() {
for _, attr := range ni.Attributes() {
switch attr.Key() {
case CapacityAttr:
n.Capacity, _ = strconv.ParseUint(attr.GetValue(), 10, 64)
n.Capacity, _ = strconv.ParseUint(attr.Value(), 10, 64)
case PriceAttr:
n.Price, _ = strconv.ParseUint(attr.GetValue(), 10, 64)
n.Price, _ = strconv.ParseUint(attr.Value(), 10, 64)
}
n.AttrMap[attr.GetKey()] = attr.GetValue()
n.AttrMap[attr.Key()] = attr.Value()
}
return n
}

View file

@ -7,19 +7,6 @@ import (
"github.com/stretchr/testify/require"
)
func TestNode_NetworkAddress(t *testing.T) {
addr := "127.0.0.1:8080"
nV2 := new(netmap.NodeInfo)
nV2.SetAddress(addr)
n := Node{
InfoV2: nV2,
}
require.Equal(t, addr, n.NetworkAddress())
}
func TestNodeStateFromV2(t *testing.T) {
for _, item := range []struct {
s NodeState

View file

@ -12,48 +12,48 @@ import (
type Selector netmap.Selector
// processSelectors processes selectors and returns error is any of them is invalid.
func (c *Context) processSelectors(p *netmap.PlacementPolicy) error {
for _, s := range p.GetSelectors() {
func (c *Context) processSelectors(p *PlacementPolicy) error {
for _, s := range p.Selectors() {
if s == nil {
return fmt.Errorf("%w: SELECT", ErrMissingField)
} else if s.GetFilter() != MainFilterName {
_, ok := c.Filters[s.GetFilter()]
} else if s.Filter() != MainFilterName {
_, ok := c.Filters[s.Filter()]
if !ok {
return fmt.Errorf("%w: SELECT FROM '%s'", ErrFilterNotFound, s.GetFilter())
return fmt.Errorf("%w: SELECT FROM '%s'", ErrFilterNotFound, s.Filter())
}
}
c.Selectors[s.GetName()] = s
c.Selectors[s.Name()] = s
result, err := c.getSelection(p, s)
if err != nil {
return err
}
c.Selections[s.GetName()] = result
c.Selections[s.Name()] = result
}
return nil
}
// GetNodesCount returns amount of buckets and nodes in every bucket
// for a given placement policy.
func GetNodesCount(p *netmap.PlacementPolicy, s *netmap.Selector) (int, int) {
switch s.GetClause() {
case netmap.Same:
return 1, int(p.GetContainerBackupFactor() * s.GetCount())
func GetNodesCount(p *PlacementPolicy, s *Selector) (int, int) {
switch s.Clause() {
case ClauseSame:
return 1, int(p.ContainerBackupFactor() * s.Count())
default:
return int(s.GetCount()), int(p.GetContainerBackupFactor())
return int(s.Count()), int(p.ContainerBackupFactor())
}
}
// getSelection returns nodes grouped by s.attribute.
func (c *Context) getSelection(p *netmap.PlacementPolicy, s *netmap.Selector) ([]Nodes, error) {
func (c *Context) getSelection(p *PlacementPolicy, s *Selector) ([]Nodes, error) {
bucketCount, nodesInBucket := GetNodesCount(p, s)
buckets := c.getSelectionBase(s)
if len(buckets) < bucketCount {
return nil, fmt.Errorf("%w: '%s'", ErrNotEnoughNodes, s.GetName())
return nil, fmt.Errorf("%w: '%s'", ErrNotEnoughNodes, s.Name())
}
if len(c.pivot) == 0 {
// Deterministic order in case of zero seed.
if s.GetAttribute() == "" {
if s.Attribute() == "" {
sort.Slice(buckets, func(i, j int) bool {
return buckets[i].nodes[0].ID < buckets[j].nodes[0].ID
})
@ -72,7 +72,7 @@ func (c *Context) getSelection(p *netmap.PlacementPolicy, s *netmap.Selector) ([
}
}
if len(nodes) < bucketCount {
return nil, fmt.Errorf("%w: '%s'", ErrNotEnoughNodes, s.GetName())
return nil, fmt.Errorf("%w: '%s'", ErrNotEnoughNodes, s.Name())
}
if len(c.pivot) != 0 {
weights := make([]float64, len(nodes))
@ -91,12 +91,12 @@ type nodeAttrPair struct {
// getSelectionBase returns nodes grouped by selector attribute.
// It it guaranteed that each pair will contain at least one node.
func (c *Context) getSelectionBase(s *netmap.Selector) []nodeAttrPair {
f := c.Filters[s.GetFilter()]
isMain := s.GetFilter() == MainFilterName
func (c *Context) getSelectionBase(s *Selector) []nodeAttrPair {
f := c.Filters[s.Filter()]
isMain := s.Filter() == MainFilterName
result := []nodeAttrPair{}
nodeMap := map[string]Nodes{}
attr := s.GetAttribute()
attr := s.Attribute()
for i := range c.Netmap.Nodes {
if isMain || c.match(f, c.Netmap.Nodes[i]) {
if attr == "" {

View file

@ -11,20 +11,20 @@ import (
func TestPlacementPolicy_UnspecifiedClause(t *testing.T) {
p := newPlacementPolicy(1,
[]*netmap.Replica{newReplica(1, "X")},
[]*netmap.Selector{
newSelector("X", "", netmap.Distinct, 4, "*"),
[]*Replica{newReplica(1, "X")},
[]*Selector{
newSelector("X", "", ClauseDistinct, 4, "*"),
},
nil,
)
nodes := []netmap.NodeInfo{
nodes := []NodeInfo{
nodeInfoFromAttributes("ID", "1", "Country", "RU", "City", "St.Petersburg", "SSD", "0"),
nodeInfoFromAttributes("ID", "2", "Country", "RU", "City", "St.Petersburg", "SSD", "1"),
nodeInfoFromAttributes("ID", "3", "Country", "RU", "City", "Moscow", "SSD", "1"),
nodeInfoFromAttributes("ID", "4", "Country", "RU", "City", "Moscow", "SSD", "1"),
}
nm, err := NewNetmap(NodesFromV2(nodes))
nm, err := NewNetmap(NodesFromInfo(nodes))
require.NoError(t, err)
v, err := nm.GetContainerNodes(p, nil)
require.NoError(t, err)
@ -33,24 +33,24 @@ func TestPlacementPolicy_UnspecifiedClause(t *testing.T) {
func TestPlacementPolicy_GetPlacementVectors(t *testing.T) {
p := newPlacementPolicy(2,
[]*netmap.Replica{
[]*Replica{
newReplica(1, "SPB"),
newReplica(2, "Americas"),
},
[]*netmap.Selector{
newSelector("SPB", "City", netmap.Same, 1, "SPBSSD"),
newSelector("Americas", "City", netmap.Distinct, 2, "Americas"),
[]*Selector{
newSelector("SPB", "City", ClauseSame, 1, "SPBSSD"),
newSelector("Americas", "City", ClauseDistinct, 2, "Americas"),
},
[]*netmap.Filter{
newFilter("SPBSSD", "", "", netmap.AND,
newFilter("", "Country", "RU", netmap.EQ),
newFilter("", "City", "St.Petersburg", netmap.EQ),
newFilter("", "SSD", "1", netmap.EQ)),
newFilter("Americas", "", "", netmap.OR,
newFilter("", "Continent", "NA", netmap.EQ),
newFilter("", "Continent", "SA", netmap.EQ)),
[]*Filter{
newFilter("SPBSSD", "", "", OpAND,
newFilter("", "Country", "RU", OpEQ),
newFilter("", "City", "St.Petersburg", OpEQ),
newFilter("", "SSD", "1", OpEQ)),
newFilter("Americas", "", "", OpOR,
newFilter("", "Continent", "NA", OpEQ),
newFilter("", "Continent", "SA", OpEQ)),
})
nodes := []netmap.NodeInfo{
nodes := []NodeInfo{
nodeInfoFromAttributes("ID", "1", "Country", "RU", "City", "St.Petersburg", "SSD", "0"),
nodeInfoFromAttributes("ID", "2", "Country", "RU", "City", "St.Petersburg", "SSD", "1"),
nodeInfoFromAttributes("ID", "3", "Country", "RU", "City", "Moscow", "SSD", "1"),
@ -66,7 +66,7 @@ func TestPlacementPolicy_GetPlacementVectors(t *testing.T) {
nodeInfoFromAttributes("ID", "13", "Continent", "SA", "City", "Lima"),
}
nm, err := NewNetmap(NodesFromV2(nodes))
nm, err := NewNetmap(NodesFromInfo(nodes))
require.NoError(t, err)
v, err := nm.GetContainerNodes(p, nil)
require.NoError(t, err)
@ -94,17 +94,17 @@ func TestPlacementPolicy_GetPlacementVectors(t *testing.T) {
func TestPlacementPolicy_ProcessSelectors(t *testing.T) {
p := newPlacementPolicy(2, nil,
[]*netmap.Selector{
newSelector("SameRU", "City", netmap.Same, 2, "FromRU"),
newSelector("DistinctRU", "City", netmap.Distinct, 2, "FromRU"),
newSelector("Good", "Country", netmap.Distinct, 2, "Good"),
newSelector("Main", "Country", netmap.Distinct, 3, "*"),
[]*Selector{
newSelector("SameRU", "City", ClauseSame, 2, "FromRU"),
newSelector("DistinctRU", "City", ClauseDistinct, 2, "FromRU"),
newSelector("Good", "Country", ClauseDistinct, 2, "Good"),
newSelector("Main", "Country", ClauseDistinct, 3, "*"),
},
[]*netmap.Filter{
newFilter("FromRU", "Country", "Russia", netmap.EQ),
newFilter("Good", "Rating", "4", netmap.GE),
[]*Filter{
newFilter("FromRU", "Country", "Russia", OpEQ),
newFilter("Good", "Rating", "4", OpGE),
})
nodes := []netmap.NodeInfo{
nodes := []NodeInfo{
nodeInfoFromAttributes("Country", "Russia", "Rating", "1", "City", "SPB"),
nodeInfoFromAttributes("Country", "Germany", "Rating", "5", "City", "Berlin"),
nodeInfoFromAttributes("Country", "Russia", "Rating", "6", "City", "Moscow"),
@ -118,22 +118,22 @@ func TestPlacementPolicy_ProcessSelectors(t *testing.T) {
nodeInfoFromAttributes("Country", "Russia", "Rating", "9", "City", "SPB"),
}
nm, err := NewNetmap(NodesFromV2(nodes))
nm, err := NewNetmap(NodesFromInfo(nodes))
require.NoError(t, err)
c := NewContext(nm)
require.NoError(t, c.processFilters(p))
require.NoError(t, c.processSelectors(p))
for _, s := range p.GetSelectors() {
sel := c.Selections[s.GetName()]
s := c.Selectors[s.GetName()]
for _, s := range p.Selectors() {
sel := c.Selections[s.Name()]
s := c.Selectors[s.Name()]
bucketCount, nodesInBucket := GetNodesCount(p, s)
targ := fmt.Sprintf("selector '%s'", s.GetName())
targ := fmt.Sprintf("selector '%s'", s.Name())
require.Equal(t, bucketCount, len(sel), targ)
for _, res := range sel {
require.Equal(t, nodesInBucket, len(res), targ)
for j := range res {
require.True(t, c.applyFilter(s.GetFilter(), res[j]), targ)
require.True(t, c.applyFilter(s.Filter(), res[j]), targ)
}
}
}
@ -142,12 +142,12 @@ func TestPlacementPolicy_ProcessSelectors(t *testing.T) {
func TestPlacementPolicy_ProcessSelectorsHRW(t *testing.T) {
p := newPlacementPolicy(1, nil,
[]*netmap.Selector{
newSelector("Main", "Country", netmap.Distinct, 3, "*"),
[]*Selector{
newSelector("Main", "Country", ClauseDistinct, 3, "*"),
}, nil)
// bucket weight order: RU > DE > FR
nodes := []netmap.NodeInfo{
nodes := []NodeInfo{
nodeInfoFromAttributes("Country", "Germany", PriceAttr, "2", CapacityAttr, "10000"),
nodeInfoFromAttributes("Country", "Germany", PriceAttr, "4", CapacityAttr, "1"),
nodeInfoFromAttributes("Country", "France", PriceAttr, "3", CapacityAttr, "10"),
@ -158,7 +158,7 @@ func TestPlacementPolicy_ProcessSelectorsHRW(t *testing.T) {
nodeInfoFromAttributes("Country", "France", PriceAttr, "7", CapacityAttr, "10000"),
nodeInfoFromAttributes("Country", "Russia", PriceAttr, "2", CapacityAttr, "1"),
}
nm, err := NewNetmap(NodesFromV2(nodes))
nm, err := NewNetmap(NodesFromInfo(nodes))
require.NoError(t, err)
c := NewContext(nm)
c.setPivot([]byte("containerID"))
@ -189,46 +189,46 @@ func TestPlacementPolicy_ProcessSelectorsHRW(t *testing.T) {
func TestPlacementPolicy_ProcessSelectorsInvalid(t *testing.T) {
testCases := []struct {
name string
p *netmap.PlacementPolicy
p *PlacementPolicy
err error
}{
{
"MissingSelector",
newPlacementPolicy(2, nil,
[]*netmap.Selector{nil},
[]*netmap.Filter{}),
[]*Selector{nil},
[]*Filter{}),
ErrMissingField,
},
{
"InvalidFilterReference",
newPlacementPolicy(1, nil,
[]*netmap.Selector{newSelector("MyStore", "Country", netmap.Distinct, 1, "FromNL")},
[]*netmap.Filter{newFilter("FromRU", "Country", "Russia", netmap.EQ)}),
[]*Selector{newSelector("MyStore", "Country", ClauseDistinct, 1, "FromNL")},
[]*Filter{newFilter("FromRU", "Country", "Russia", OpEQ)}),
ErrFilterNotFound,
},
{
"NotEnoughNodes (backup factor)",
newPlacementPolicy(2, nil,
[]*netmap.Selector{newSelector("MyStore", "Country", netmap.Distinct, 1, "FromRU")},
[]*netmap.Filter{newFilter("FromRU", "Country", "Russia", netmap.EQ)}),
[]*Selector{newSelector("MyStore", "Country", ClauseDistinct, 1, "FromRU")},
[]*Filter{newFilter("FromRU", "Country", "Russia", OpEQ)}),
ErrNotEnoughNodes,
},
{
"NotEnoughNodes (buckets)",
newPlacementPolicy(1, nil,
[]*netmap.Selector{newSelector("MyStore", "Country", netmap.Distinct, 2, "FromRU")},
[]*netmap.Filter{newFilter("FromRU", "Country", "Russia", netmap.EQ)}),
[]*Selector{newSelector("MyStore", "Country", ClauseDistinct, 2, "FromRU")},
[]*Filter{newFilter("FromRU", "Country", "Russia", OpEQ)}),
ErrNotEnoughNodes,
},
}
nodes := []netmap.NodeInfo{
nodes := []NodeInfo{
nodeInfoFromAttributes("Country", "Russia"),
nodeInfoFromAttributes("Country", "Germany"),
nodeInfoFromAttributes(),
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
nm, err := NewNetmap(NodesFromV2(nodes))
nm, err := NewNetmap(NodesFromInfo(nodes))
require.NoError(t, err)
c := NewContext(nm)
require.NoError(t, c.processFilters(tc.p))