Move to frostfs-node

Signed-off-by: Pavel Karpy <p.karpy@yadro.com>
This commit is contained in:
Pavel Karpy 2022-12-23 20:35:35 +03:00 committed by Stanislav Bogatyrev
parent 42554a9298
commit 923f84722a
934 changed files with 3470 additions and 3451 deletions

View file

@ -0,0 +1,34 @@
package main
import (
accountingGRPC "github.com/TrueCloudLab/frostfs-api-go/v2/accounting/grpc"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/client/balance"
accountingTransportGRPC "github.com/TrueCloudLab/frostfs-node/pkg/network/transport/accounting/grpc"
accountingService "github.com/TrueCloudLab/frostfs-node/pkg/services/accounting"
accounting "github.com/TrueCloudLab/frostfs-node/pkg/services/accounting/morph"
)
func initAccountingService(c *cfg) {
if c.cfgMorph.client == nil {
initMorphComponents(c)
}
balanceMorphWrapper, err := balance.NewFromMorph(c.cfgMorph.client, c.cfgAccounting.scriptHash, 0)
fatalOnErr(err)
server := accountingTransportGRPC.New(
accountingService.NewSignService(
&c.key.PrivateKey,
accountingService.NewResponseService(
accountingService.NewExecutionService(
accounting.NewExecutor(balanceMorphWrapper),
),
c.respSvc,
),
),
)
for _, srv := range c.cfgGRPC.servers {
accountingGRPC.RegisterAccountingServiceServer(srv, server)
}
}

View file

@ -0,0 +1,14 @@
package main
import (
nodeconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/node"
"github.com/TrueCloudLab/frostfs-node/pkg/util/attributes"
)
func parseAttributes(c *cfg) {
if nodeconfig.Relay(c.appCfg) {
return
}
fatalOnErr(attributes.ReadNodeAttributes(&c.cfgNodeInfo.localInfo, nodeconfig.Attributes(c.appCfg)))
}

423
cmd/frostfs-node/cache.go Normal file
View file

@ -0,0 +1,423 @@
package main
import (
"sync"
"time"
"github.com/TrueCloudLab/frostfs-node/pkg/core/container"
"github.com/TrueCloudLab/frostfs-node/pkg/core/netmap"
cntClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client/container"
putsvc "github.com/TrueCloudLab/frostfs-node/pkg/services/object/put"
apistatus "github.com/TrueCloudLab/frostfs-sdk-go/client/status"
cid "github.com/TrueCloudLab/frostfs-sdk-go/container/id"
netmapSDK "github.com/TrueCloudLab/frostfs-sdk-go/netmap"
"github.com/TrueCloudLab/frostfs-sdk-go/user"
lru "github.com/hashicorp/golang-lru"
)
type netValueReader func(interface{}) (interface{}, error)
type valueWithTime struct {
v interface{}
t time.Time
// cached error in order to not repeat failed request for some time
e error
}
// entity that provides TTL cache interface.
type ttlNetCache struct {
ttl time.Duration
sz int
cache *lru.Cache
netRdr netValueReader
}
// complicates netValueReader with TTL caching mechanism.
func newNetworkTTLCache(sz int, ttl time.Duration, netRdr netValueReader) *ttlNetCache {
cache, err := lru.New(sz)
fatalOnErr(err)
return &ttlNetCache{
ttl: ttl,
sz: sz,
cache: cache,
netRdr: netRdr,
}
}
// reads value by the key.
//
// updates the value from the network on cache miss or by TTL.
//
// returned value should not be modified.
func (c *ttlNetCache) get(key interface{}) (interface{}, error) {
val, ok := c.cache.Peek(key)
if ok {
valWithTime := val.(*valueWithTime)
if time.Since(valWithTime.t) < c.ttl {
return valWithTime.v, valWithTime.e
}
c.cache.Remove(key)
}
val, err := c.netRdr(key)
c.set(key, val, err)
return val, err
}
func (c *ttlNetCache) set(k, v interface{}, e error) {
c.cache.Add(k, &valueWithTime{
v: v,
t: time.Now(),
e: e,
})
}
func (c *ttlNetCache) remove(key interface{}) {
c.cache.Remove(key)
}
// entity that provides LRU cache interface.
type lruNetCache struct {
cache *lru.Cache
netRdr netValueReader
}
// newNetworkLRUCache returns wrapper over netValueReader with LRU cache.
func newNetworkLRUCache(sz int, netRdr netValueReader) *lruNetCache {
cache, err := lru.New(sz)
fatalOnErr(err)
return &lruNetCache{
cache: cache,
netRdr: netRdr,
}
}
// reads value by the key.
//
// updates the value from the network on cache miss.
//
// returned value should not be modified.
func (c *lruNetCache) get(key interface{}) (interface{}, error) {
val, ok := c.cache.Get(key)
if ok {
return val, nil
}
val, err := c.netRdr(key)
if err != nil {
return nil, err
}
c.cache.Add(key, val)
return val, nil
}
// wrapper over TTL cache of values read from the network
// that implements container storage.
type ttlContainerStorage ttlNetCache
func newCachedContainerStorage(v container.Source, ttl time.Duration) *ttlContainerStorage {
const containerCacheSize = 100
lruCnrCache := newNetworkTTLCache(containerCacheSize, ttl, func(key interface{}) (interface{}, error) {
var id cid.ID
err := id.DecodeString(key.(string))
if err != nil {
return nil, err
}
return v.Get(id)
})
return (*ttlContainerStorage)(lruCnrCache)
}
func (s *ttlContainerStorage) handleRemoval(cnr cid.ID) {
(*ttlNetCache)(s).set(cnr.EncodeToString(), nil, apistatus.ContainerNotFound{})
}
// Get returns container value from the cache. If value is missing in the cache
// or expired, then it returns value from side chain and updates the cache.
func (s *ttlContainerStorage) Get(cnr cid.ID) (*container.Container, error) {
val, err := (*ttlNetCache)(s).get(cnr.EncodeToString())
if err != nil {
return nil, err
}
return val.(*container.Container), nil
}
type ttlEACLStorage ttlNetCache
func newCachedEACLStorage(v container.EACLSource, ttl time.Duration) *ttlEACLStorage {
const eaclCacheSize = 100
lruCnrCache := newNetworkTTLCache(eaclCacheSize, ttl, func(key interface{}) (interface{}, error) {
var id cid.ID
err := id.DecodeString(key.(string))
if err != nil {
return nil, err
}
return v.GetEACL(id)
})
return (*ttlEACLStorage)(lruCnrCache)
}
// GetEACL returns eACL value from the cache. If value is missing in the cache
// or expired, then it returns value from side chain and updates cache.
func (s *ttlEACLStorage) GetEACL(cnr cid.ID) (*container.EACL, error) {
val, err := (*ttlNetCache)(s).get(cnr.EncodeToString())
if err != nil {
return nil, err
}
return val.(*container.EACL), nil
}
// InvalidateEACL removes cached eACL value.
func (s *ttlEACLStorage) InvalidateEACL(cnr cid.ID) {
(*ttlNetCache)(s).remove(cnr.EncodeToString())
}
type lruNetmapSource struct {
netState netmap.State
cache *lruNetCache
}
func newCachedNetmapStorage(s netmap.State, v netmap.Source) netmap.Source {
const netmapCacheSize = 10
lruNetmapCache := newNetworkLRUCache(netmapCacheSize, func(key interface{}) (interface{}, error) {
return v.GetNetMapByEpoch(key.(uint64))
})
return &lruNetmapSource{
netState: s,
cache: lruNetmapCache,
}
}
func (s *lruNetmapSource) GetNetMap(diff uint64) (*netmapSDK.NetMap, error) {
return s.getNetMapByEpoch(s.netState.CurrentEpoch() - diff)
}
func (s *lruNetmapSource) GetNetMapByEpoch(epoch uint64) (*netmapSDK.NetMap, error) {
return s.getNetMapByEpoch(epoch)
}
func (s *lruNetmapSource) getNetMapByEpoch(epoch uint64) (*netmapSDK.NetMap, error) {
val, err := s.cache.get(epoch)
if err != nil {
return nil, err
}
return val.(*netmapSDK.NetMap), nil
}
func (s *lruNetmapSource) Epoch() (uint64, error) {
return s.netState.CurrentEpoch(), nil
}
// wrapper over TTL cache of values read from the network
// that implements container lister.
type ttlContainerLister ttlNetCache
// value type for ttlNetCache used by ttlContainerLister.
type cacheItemContainerList struct {
// protects list from concurrent add/remove ops
mtx sync.RWMutex
// actual list of containers owner by the particular user
list []cid.ID
}
func newCachedContainerLister(c *cntClient.Client, ttl time.Duration) *ttlContainerLister {
const containerListerCacheSize = 100
lruCnrListerCache := newNetworkTTLCache(containerListerCacheSize, ttl, func(key interface{}) (interface{}, error) {
var (
id *user.ID
strID = key.(string)
)
if strID != "" {
id = new(user.ID)
err := id.DecodeString(strID)
if err != nil {
return nil, err
}
}
list, err := c.List(id)
if err != nil {
return nil, err
}
return &cacheItemContainerList{
list: list,
}, nil
})
return (*ttlContainerLister)(lruCnrListerCache)
}
// List returns list of container IDs from the cache. If list is missing in the
// cache or expired, then it returns container IDs from side chain and updates
// the cache.
func (s *ttlContainerLister) List(id *user.ID) ([]cid.ID, error) {
var str string
if id != nil {
str = id.EncodeToString()
}
val, err := (*ttlNetCache)(s).get(str)
if err != nil {
return nil, err
}
// panic on typecast below is OK since developer must be careful,
// runtime can do nothing with wrong type occurrence
item := val.(*cacheItemContainerList)
item.mtx.RLock()
res := make([]cid.ID, len(item.list))
copy(res, item.list)
item.mtx.RUnlock()
return res, nil
}
// updates cached list of owner's containers: cnr is added if flag is true, otherwise it's removed.
// Concurrent calls can lead to some races:
// - two parallel additions to missing owner's cache can lead to only one container to be cached
// - async cache value eviction can lead to idle addition
//
// All described race cases aren't critical since cache values expire anyway, we just try
// to increase cache actuality w/o huge overhead on synchronization.
func (s *ttlContainerLister) update(owner user.ID, cnr cid.ID, add bool) {
strOwner := owner.EncodeToString()
val, ok := (*ttlNetCache)(s).cache.Get(strOwner)
if !ok {
// we could cache the single cnr but in this case we will disperse
// with the Sidechain a lot
return
}
// panic on typecast below is OK since developer must be careful,
// runtime can do nothing with wrong type occurrence
item := val.(*valueWithTime).v.(*cacheItemContainerList)
item.mtx.Lock()
{
found := false
for i := range item.list {
if found = item.list[i].Equals(cnr); found {
if !add {
item.list = append(item.list[:i], item.list[i+1:]...)
// if list became empty we don't remove the value from the cache
// since empty list is a correct value, and we don't want to insta
// re-request it from the Sidechain
}
break
}
}
if add && !found {
item.list = append(item.list, cnr)
}
}
item.mtx.Unlock()
}
type cachedIRFetcher ttlNetCache
func newCachedIRFetcher(f interface{ InnerRingKeys() ([][]byte, error) }) *cachedIRFetcher {
const (
irFetcherCacheSize = 1 // we intend to store only one value
// Without the cache in the testnet we can see several hundred simultaneous
// requests (frostfs-node #1278), so limiting the request rate solves the issue.
//
// Exact request rate doesn't really matter because Inner Ring list update
// happens extremely rare, but there is no side chain events for that as
// for now (frostfs-contract v0.15.0 notary disabled env) to monitor it.
irFetcherCacheTTL = 30 * time.Second
)
irFetcherCache := newNetworkTTLCache(irFetcherCacheSize, irFetcherCacheTTL,
func(key interface{}) (interface{}, error) {
return f.InnerRingKeys()
},
)
return (*cachedIRFetcher)(irFetcherCache)
}
// InnerRingKeys returns cached list of Inner Ring keys. If keys are missing in
// the cache or expired, then it returns keys from side chain and updates
// the cache.
func (f *cachedIRFetcher) InnerRingKeys() ([][]byte, error) {
val, err := (*ttlNetCache)(f).get("")
if err != nil {
return nil, err
}
return val.([][]byte), nil
}
type ttlMaxObjectSizeCache struct {
mtx sync.RWMutex
lastUpdated time.Time
lastSize uint64
src putsvc.MaxSizeSource
}
func newCachedMaxObjectSizeSource(src putsvc.MaxSizeSource) putsvc.MaxSizeSource {
return &ttlMaxObjectSizeCache{
src: src,
}
}
func (c *ttlMaxObjectSizeCache) MaxObjectSize() uint64 {
const ttl = time.Second * 30
c.mtx.RLock()
prevUpdated := c.lastUpdated
size := c.lastSize
c.mtx.RUnlock()
if time.Since(prevUpdated) < ttl {
return size
}
c.mtx.Lock()
size = c.lastSize
if !c.lastUpdated.After(prevUpdated) {
size = c.src.MaxObjectSize()
c.lastSize = size
c.lastUpdated = time.Now()
}
c.mtx.Unlock()
return size
}

969
cmd/frostfs-node/config.go Normal file
View file

@ -0,0 +1,969 @@
package main
import (
"context"
"errors"
"fmt"
"io/fs"
"net"
"os"
"os/signal"
"path/filepath"
"strings"
"sync"
atomicstd "sync/atomic"
"syscall"
"time"
netmapV2 "github.com/TrueCloudLab/frostfs-api-go/v2/netmap"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
apiclientconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/apiclient"
contractsconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/contracts"
engineconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine"
shardconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard"
blobovniczaconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor/blobovnicza"
fstreeconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor/fstree"
loggerconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/logger"
metricsconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/metrics"
nodeconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/node"
objectconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/object"
replicatorconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/replicator"
"github.com/TrueCloudLab/frostfs-node/pkg/core/container"
netmapCore "github.com/TrueCloudLab/frostfs-node/pkg/core/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/blobovniczatree"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/fstree"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/engine"
meta "github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/pilorama"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
shardmode "github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache"
"github.com/TrueCloudLab/frostfs-node/pkg/metrics"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/client"
containerClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client/container"
nmClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/event"
netmap2 "github.com/TrueCloudLab/frostfs-node/pkg/morph/event/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/network"
"github.com/TrueCloudLab/frostfs-node/pkg/network/cache"
"github.com/TrueCloudLab/frostfs-node/pkg/services/control"
getsvc "github.com/TrueCloudLab/frostfs-node/pkg/services/object/get"
"github.com/TrueCloudLab/frostfs-node/pkg/services/object_manager/tombstone"
tsourse "github.com/TrueCloudLab/frostfs-node/pkg/services/object_manager/tombstone/source"
"github.com/TrueCloudLab/frostfs-node/pkg/services/replicator"
trustcontroller "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/local/controller"
truststorage "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/local/storage"
"github.com/TrueCloudLab/frostfs-node/pkg/services/tree"
"github.com/TrueCloudLab/frostfs-node/pkg/services/util/response"
"github.com/TrueCloudLab/frostfs-node/pkg/util"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
"github.com/TrueCloudLab/frostfs-node/pkg/util/state"
"github.com/TrueCloudLab/frostfs-sdk-go/netmap"
objectSDK "github.com/TrueCloudLab/frostfs-sdk-go/object"
"github.com/TrueCloudLab/frostfs-sdk-go/user"
"github.com/TrueCloudLab/frostfs-sdk-go/version"
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
neogoutil "github.com/nspcc-dev/neo-go/pkg/util"
"github.com/panjf2000/ants/v2"
"go.etcd.io/bbolt"
"go.uber.org/atomic"
"go.uber.org/zap"
"google.golang.org/grpc"
)
const addressSize = 72 // 32 bytes object ID, 32 bytes container ID, 8 bytes protobuf encoding
const maxMsgSize = 4 << 20 // transport msg limit 4 MiB
// capacity of the pools of the morph notification handlers
// for each contract listener.
const notificationHandlerPoolSize = 10
// applicationConfiguration reads and stores component-specific configuration
// values. It should not store any application helpers structs (pointers to shared
// structs).
// It must not be used concurrently.
type applicationConfiguration struct {
// _read indicated whether a config
// has already been read
_read bool
LoggerCfg struct {
level string
}
EngineCfg struct {
errorThreshold uint32
shardPoolSize uint32
shards []shardCfg
}
}
type shardCfg struct {
compress bool
smallSizeObjectLimit uint64
uncompressableContentType []string
refillMetabase bool
mode shardmode.Mode
metaCfg struct {
path string
perm fs.FileMode
maxBatchSize int
maxBatchDelay time.Duration
}
subStorages []subStorageCfg
gcCfg struct {
removerBatchSize int
removerSleepInterval time.Duration
}
writecacheCfg struct {
enabled bool
path string
maxBatchSize int
maxBatchDelay time.Duration
smallObjectSize uint64
maxObjSize uint64
flushWorkerCount int
sizeLimit uint64
noSync bool
}
piloramaCfg struct {
enabled bool
path string
perm fs.FileMode
noSync bool
maxBatchSize int
maxBatchDelay time.Duration
}
}
// id returns persistent id of a shard. It is different from the ID used in runtime
// and is primarily used to identify shards in the configuration.
func (c *shardCfg) id() string {
// This calculation should be kept in sync with
// pkg/local_object_storage/engine/control.go file.
var sb strings.Builder
for i := range c.subStorages {
sb.WriteString(filepath.Clean(c.subStorages[i].path))
}
return sb.String()
}
type subStorageCfg struct {
// common for all storages
typ string
path string
perm fs.FileMode
depth uint64
noSync bool
// blobovnicza-specific
size uint64
width uint64
openedCacheSize int
}
// readConfig fills applicationConfiguration with raw configuration values
// not modifying them.
func (a *applicationConfiguration) readConfig(c *config.Config) error {
if a._read {
err := c.Reload()
if err != nil {
return fmt.Errorf("could not reload configuration: %w", err)
}
err = validateConfig(c)
if err != nil {
return fmt.Errorf("configuration's validation: %w", err)
}
// clear if it is rereading
*a = applicationConfiguration{}
}
a._read = true
// Logger
a.LoggerCfg.level = loggerconfig.Level(c)
// Storage Engine
a.EngineCfg.errorThreshold = engineconfig.ShardErrorThreshold(c)
a.EngineCfg.shardPoolSize = engineconfig.ShardPoolSize(c)
return engineconfig.IterateShards(c, false, func(sc *shardconfig.Config) error {
var sh shardCfg
sh.refillMetabase = sc.RefillMetabase()
sh.mode = sc.Mode()
sh.compress = sc.Compress()
sh.uncompressableContentType = sc.UncompressableContentTypes()
sh.smallSizeObjectLimit = sc.SmallSizeLimit()
// write-cache
writeCacheCfg := sc.WriteCache()
if writeCacheCfg.Enabled() {
wc := &sh.writecacheCfg
wc.enabled = true
wc.path = writeCacheCfg.Path()
wc.maxBatchSize = writeCacheCfg.BoltDB().MaxBatchSize()
wc.maxBatchDelay = writeCacheCfg.BoltDB().MaxBatchDelay()
wc.maxObjSize = writeCacheCfg.MaxObjectSize()
wc.smallObjectSize = writeCacheCfg.SmallObjectSize()
wc.flushWorkerCount = writeCacheCfg.WorkersNumber()
wc.sizeLimit = writeCacheCfg.SizeLimit()
wc.noSync = writeCacheCfg.NoSync()
}
// blobstor with substorages
blobStorCfg := sc.BlobStor()
storagesCfg := blobStorCfg.Storages()
metabaseCfg := sc.Metabase()
gcCfg := sc.GC()
if config.BoolSafe(c.Sub("tree"), "enabled") {
piloramaCfg := sc.Pilorama()
pr := &sh.piloramaCfg
pr.enabled = true
pr.path = piloramaCfg.Path()
pr.perm = piloramaCfg.Perm()
pr.noSync = piloramaCfg.NoSync()
pr.maxBatchSize = piloramaCfg.MaxBatchSize()
pr.maxBatchDelay = piloramaCfg.MaxBatchDelay()
}
ss := make([]subStorageCfg, 0, len(storagesCfg))
for i := range storagesCfg {
var sCfg subStorageCfg
sCfg.typ = storagesCfg[i].Type()
sCfg.path = storagesCfg[i].Path()
sCfg.perm = storagesCfg[i].Perm()
switch storagesCfg[i].Type() {
case blobovniczatree.Type:
sub := blobovniczaconfig.From((*config.Config)(storagesCfg[i]))
sCfg.size = sub.Size()
sCfg.depth = sub.ShallowDepth()
sCfg.width = sub.ShallowWidth()
sCfg.openedCacheSize = sub.OpenedCacheSize()
case fstree.Type:
sub := fstreeconfig.From((*config.Config)(storagesCfg[i]))
sCfg.depth = sub.Depth()
sCfg.noSync = sub.NoSync()
default:
return fmt.Errorf("invalid storage type: %s", storagesCfg[i].Type())
}
ss = append(ss, sCfg)
}
sh.subStorages = ss
// meta
m := &sh.metaCfg
m.path = metabaseCfg.Path()
m.perm = metabaseCfg.BoltDB().Perm()
m.maxBatchDelay = metabaseCfg.BoltDB().MaxBatchDelay()
m.maxBatchSize = metabaseCfg.BoltDB().MaxBatchSize()
// GC
sh.gcCfg.removerBatchSize = gcCfg.RemoverBatchSize()
sh.gcCfg.removerSleepInterval = gcCfg.RemoverSleepInterval()
a.EngineCfg.shards = append(a.EngineCfg.shards, sh)
return nil
})
}
// internals contains application-specific internals that are created
// on application startup and are shared b/w the components during
// the application life cycle.
// It should not contain any read configuration values, component-specific
// helpers and fields.
type internals struct {
ctx context.Context
ctxCancel func()
internalErr chan error // channel for internal application errors at runtime
appCfg *config.Config
log *logger.Logger
wg *sync.WaitGroup
workers []worker
closers []func()
apiVersion version.Version
healthStatus *atomic.Int32
// is node under maintenance
isMaintenance atomic.Bool
}
// starts node's maintenance.
func (c *cfg) startMaintenance() {
c.isMaintenance.Store(true)
c.cfgNetmap.state.setControlNetmapStatus(control.NetmapStatus_MAINTENANCE)
c.log.Info("started local node's maintenance")
}
// stops node's maintenance.
func (c *internals) stopMaintenance() {
c.isMaintenance.Store(false)
c.log.Info("stopped local node's maintenance")
}
// IsMaintenance checks if storage node is under maintenance.
//
// Provides util.NodeState to Object service.
func (c *internals) IsMaintenance() bool {
return c.isMaintenance.Load()
}
// shared contains component-specific structs/helpers that should
// be shared during initialization of the application.
type shared struct {
privateTokenStore sessionStorage
persistate *state.PersistentStorage
clientCache *cache.ClientCache
bgClientCache *cache.ClientCache
localAddr network.AddressGroup
key *keys.PrivateKey
binPublicKey []byte
ownerIDFromKey user.ID // user ID calculated from key
// current network map
netMap atomicstd.Value // type netmap.NetMap
netMapSource netmapCore.Source
cnrClient *containerClient.Client
respSvc *response.Service
replicator *replicator.Replicator
treeService *tree.Service
metricsCollector *metrics.NodeMetrics
}
// dynamicConfiguration stores parameters of the
// components that supports runtime reconfigurations.
type dynamicConfiguration struct {
logger *logger.Prm
}
type cfg struct {
applicationConfiguration
internals
shared
dynamicConfiguration
// configuration of the internal
// services
cfgGRPC cfgGRPC
cfgMorph cfgMorph
cfgAccounting cfgAccounting
cfgContainer cfgContainer
cfgNodeInfo cfgNodeInfo
cfgNetmap cfgNetmap
cfgControlService cfgControlService
cfgReputation cfgReputation
cfgObject cfgObject
cfgNotifications cfgNotifications
}
// ReadCurrentNetMap reads network map which has been cached at the
// latest epoch. Returns an error if value has not been cached yet.
//
// Provides interface for NetmapService server.
func (c *cfg) ReadCurrentNetMap(msg *netmapV2.NetMap) error {
val := c.netMap.Load()
if val == nil {
return errors.New("missing local network map")
}
val.(netmap.NetMap).WriteToV2(msg)
return nil
}
type cfgGRPC struct {
listeners []net.Listener
servers []*grpc.Server
maxChunkSize uint64
maxAddrAmount uint64
}
type cfgMorph struct {
client *client.Client
notaryEnabled bool
// TTL of Sidechain cached values. Non-positive value disables caching.
cacheTTL time.Duration
eigenTrustTicker *eigenTrustTickers // timers for EigenTrust iterations
proxyScriptHash neogoutil.Uint160
}
type cfgAccounting struct {
scriptHash neogoutil.Uint160
}
type cfgContainer struct {
scriptHash neogoutil.Uint160
parsers map[event.Type]event.NotificationParser
subscribers map[event.Type][]event.Handler
workerPool util.WorkerPool // pool for asynchronous handlers
}
type cfgNetmap struct {
scriptHash neogoutil.Uint160
wrapper *nmClient.Client
parsers map[event.Type]event.NotificationParser
subscribers map[event.Type][]event.Handler
workerPool util.WorkerPool // pool for asynchronous handlers
state *networkState
needBootstrap bool
reBoostrapTurnedOff *atomic.Bool // managed by control service in runtime
startEpoch uint64 // epoch number when application is started
}
type cfgNodeInfo struct {
// values from config
localInfo netmap.NodeInfo
}
type cfgObject struct {
getSvc *getsvc.Service
cnrSource container.Source
eaclSource container.EACLSource
pool cfgObjectRoutines
cfgLocalStorage cfgLocalStorage
}
type cfgNotifications struct {
enabled bool
nw notificationWriter
defaultTopic string
}
type cfgLocalStorage struct {
localStorage *engine.StorageEngine
}
type cfgObjectRoutines struct {
putRemote *ants.Pool
putRemoteCapacity int
replicatorPoolSize int
replication *ants.Pool
}
type cfgControlService struct {
server *grpc.Server
}
type cfgReputation struct {
workerPool util.WorkerPool // pool for EigenTrust algorithm's iterations
localTrustStorage *truststorage.Storage
localTrustCtrl *trustcontroller.Controller
scriptHash neogoutil.Uint160
}
var persistateSideChainLastBlockKey = []byte("side_chain_last_processed_block")
func initCfg(appCfg *config.Config) *cfg {
c := &cfg{}
err := c.readConfig(appCfg)
if err != nil {
panic(fmt.Errorf("config reading: %w", err))
}
key := nodeconfig.Key(appCfg)
logPrm, err := c.loggerPrm()
fatalOnErr(err)
log, err := logger.NewLogger(logPrm)
fatalOnErr(err)
var netAddr network.AddressGroup
relayOnly := nodeconfig.Relay(appCfg)
if !relayOnly {
netAddr = nodeconfig.BootstrapAddresses(appCfg)
}
maxChunkSize := uint64(maxMsgSize) * 3 / 4 // 25% to meta, 75% to payload
maxAddrAmount := uint64(maxChunkSize) / addressSize // each address is about 72 bytes
netState := newNetworkState()
persistate, err := state.NewPersistentStorage(nodeconfig.PersistentState(appCfg).Path())
fatalOnErr(err)
containerWorkerPool, err := ants.NewPool(notificationHandlerPoolSize)
fatalOnErr(err)
netmapWorkerPool, err := ants.NewPool(notificationHandlerPoolSize)
fatalOnErr(err)
reputationWorkerPool, err := ants.NewPool(notificationHandlerPoolSize)
fatalOnErr(err)
c.internals = internals{
ctx: context.Background(),
appCfg: appCfg,
internalErr: make(chan error),
log: log,
wg: new(sync.WaitGroup),
apiVersion: version.Current(),
healthStatus: atomic.NewInt32(int32(control.HealthStatus_HEALTH_STATUS_UNDEFINED)),
}
cacheOpts := cache.ClientCacheOpts{
DialTimeout: apiclientconfig.DialTimeout(appCfg),
StreamTimeout: apiclientconfig.StreamTimeout(appCfg),
Key: &key.PrivateKey,
AllowExternal: apiclientconfig.AllowExternal(appCfg),
}
c.shared = shared{
key: key,
binPublicKey: key.PublicKey().Bytes(),
localAddr: netAddr,
respSvc: response.NewService(response.WithNetworkState(netState)),
clientCache: cache.NewSDKClientCache(cacheOpts),
bgClientCache: cache.NewSDKClientCache(cacheOpts),
persistate: persistate,
}
c.cfgAccounting = cfgAccounting{
scriptHash: contractsconfig.Balance(appCfg),
}
c.cfgContainer = cfgContainer{
scriptHash: contractsconfig.Container(appCfg),
workerPool: containerWorkerPool,
}
c.cfgNetmap = cfgNetmap{
scriptHash: contractsconfig.Netmap(appCfg),
state: netState,
workerPool: netmapWorkerPool,
needBootstrap: !relayOnly,
reBoostrapTurnedOff: atomic.NewBool(relayOnly),
}
c.cfgGRPC = cfgGRPC{
maxChunkSize: maxChunkSize,
maxAddrAmount: maxAddrAmount,
}
c.cfgMorph = cfgMorph{
proxyScriptHash: contractsconfig.Proxy(appCfg),
}
c.cfgObject = cfgObject{
pool: initObjectPool(appCfg),
}
c.cfgReputation = cfgReputation{
scriptHash: contractsconfig.Reputation(appCfg),
workerPool: reputationWorkerPool,
}
user.IDFromKey(&c.ownerIDFromKey, key.PrivateKey.PublicKey)
if metricsconfig.Enabled(c.appCfg) {
c.metricsCollector = metrics.NewNodeMetrics()
netState.metrics = c.metricsCollector
}
c.onShutdown(c.clientCache.CloseAll) // clean up connections
c.onShutdown(c.bgClientCache.CloseAll) // clean up connections
c.onShutdown(func() { _ = c.persistate.Close() })
return c
}
func (c *cfg) engineOpts() []engine.Option {
opts := make([]engine.Option, 0, 4)
opts = append(opts,
engine.WithShardPoolSize(c.EngineCfg.shardPoolSize),
engine.WithErrorThreshold(c.EngineCfg.errorThreshold),
engine.WithLogger(c.log),
)
if c.metricsCollector != nil {
opts = append(opts, engine.WithMetrics(c.metricsCollector))
}
return opts
}
type shardOptsWithID struct {
configID string
shOpts []shard.Option
}
func (c *cfg) shardOpts() []shardOptsWithID {
shards := make([]shardOptsWithID, 0, len(c.EngineCfg.shards))
for _, shCfg := range c.EngineCfg.shards {
var writeCacheOpts []writecache.Option
if wcRead := shCfg.writecacheCfg; wcRead.enabled {
writeCacheOpts = append(writeCacheOpts,
writecache.WithPath(wcRead.path),
writecache.WithMaxBatchSize(wcRead.maxBatchSize),
writecache.WithMaxBatchDelay(wcRead.maxBatchDelay),
writecache.WithMaxObjectSize(wcRead.maxObjSize),
writecache.WithSmallObjectSize(wcRead.smallObjectSize),
writecache.WithFlushWorkersCount(wcRead.flushWorkerCount),
writecache.WithMaxCacheSize(wcRead.sizeLimit),
writecache.WithNoSync(wcRead.noSync),
writecache.WithLogger(c.log),
)
}
var piloramaOpts []pilorama.Option
if prRead := shCfg.piloramaCfg; prRead.enabled {
piloramaOpts = append(piloramaOpts,
pilorama.WithPath(prRead.path),
pilorama.WithPerm(prRead.perm),
pilorama.WithNoSync(prRead.noSync),
pilorama.WithMaxBatchSize(prRead.maxBatchSize),
pilorama.WithMaxBatchDelay(prRead.maxBatchDelay),
)
}
var ss []blobstor.SubStorage
for _, sRead := range shCfg.subStorages {
switch sRead.typ {
case blobovniczatree.Type:
ss = append(ss, blobstor.SubStorage{
Storage: blobovniczatree.NewBlobovniczaTree(
blobovniczatree.WithRootPath(sRead.path),
blobovniczatree.WithPermissions(sRead.perm),
blobovniczatree.WithBlobovniczaSize(sRead.size),
blobovniczatree.WithBlobovniczaShallowDepth(sRead.depth),
blobovniczatree.WithBlobovniczaShallowWidth(sRead.width),
blobovniczatree.WithOpenedCacheSize(sRead.openedCacheSize),
blobovniczatree.WithLogger(c.log)),
Policy: func(_ *objectSDK.Object, data []byte) bool {
return uint64(len(data)) < shCfg.smallSizeObjectLimit
},
})
case fstree.Type:
ss = append(ss, blobstor.SubStorage{
Storage: fstree.New(
fstree.WithPath(sRead.path),
fstree.WithPerm(sRead.perm),
fstree.WithDepth(sRead.depth),
fstree.WithNoSync(sRead.noSync)),
Policy: func(_ *objectSDK.Object, data []byte) bool {
return true
},
})
default:
// should never happen, that has already
// been handled: when the config was read
}
}
var sh shardOptsWithID
sh.configID = shCfg.id()
sh.shOpts = []shard.Option{
shard.WithLogger(c.log),
shard.WithRefillMetabase(shCfg.refillMetabase),
shard.WithMode(shCfg.mode),
shard.WithBlobStorOptions(
blobstor.WithCompressObjects(shCfg.compress),
blobstor.WithUncompressableContentTypes(shCfg.uncompressableContentType),
blobstor.WithStorages(ss),
blobstor.WithLogger(c.log),
),
shard.WithMetaBaseOptions(
meta.WithPath(shCfg.metaCfg.path),
meta.WithPermissions(shCfg.metaCfg.perm),
meta.WithMaxBatchSize(shCfg.metaCfg.maxBatchSize),
meta.WithMaxBatchDelay(shCfg.metaCfg.maxBatchDelay),
meta.WithBoltDBOptions(&bbolt.Options{
Timeout: 100 * time.Millisecond,
}),
meta.WithLogger(c.log),
meta.WithEpochState(c.cfgNetmap.state),
),
shard.WithPiloramaOptions(piloramaOpts...),
shard.WithWriteCache(shCfg.writecacheCfg.enabled),
shard.WithWriteCacheOptions(writeCacheOpts...),
shard.WithRemoverBatchSize(shCfg.gcCfg.removerBatchSize),
shard.WithGCRemoverSleepInterval(shCfg.gcCfg.removerSleepInterval),
shard.WithGCWorkerPoolInitializer(func(sz int) util.WorkerPool {
pool, err := ants.NewPool(sz)
fatalOnErr(err)
return pool
}),
}
shards = append(shards, sh)
}
return shards
}
func (c *cfg) loggerPrm() (*logger.Prm, error) {
// check if it has been inited before
if c.dynamicConfiguration.logger == nil {
c.dynamicConfiguration.logger = new(logger.Prm)
}
// (re)init read configuration
err := c.dynamicConfiguration.logger.SetLevelString(c.LoggerCfg.level)
if err != nil {
// not expected since validation should be performed before
panic(fmt.Sprintf("incorrect log level format: %s", c.LoggerCfg.level))
}
return c.dynamicConfiguration.logger, nil
}
func (c *cfg) LocalAddress() network.AddressGroup {
return c.localAddr
}
func initLocalStorage(c *cfg) {
ls := engine.New(c.engineOpts()...)
addNewEpochAsyncNotificationHandler(c, func(ev event.Event) {
ls.HandleNewEpoch(ev.(netmap2.NewEpoch).EpochNumber())
})
// allocate memory for the service;
// service will be created later
c.cfgObject.getSvc = new(getsvc.Service)
var tssPrm tsourse.TombstoneSourcePrm
tssPrm.SetGetService(c.cfgObject.getSvc)
tombstoneSrc := tsourse.NewSource(tssPrm)
tombstoneSource := tombstone.NewChecker(
tombstone.WithLogger(c.log),
tombstone.WithTombstoneSource(tombstoneSrc),
)
for _, optsWithMeta := range c.shardOpts() {
id, err := ls.AddShard(append(optsWithMeta.shOpts, shard.WithTombstoneSource(tombstoneSource))...)
fatalOnErr(err)
c.log.Info("shard attached to engine",
zap.Stringer("id", id),
)
}
c.cfgObject.cfgLocalStorage.localStorage = ls
c.onShutdown(func() {
c.log.Info("closing components of the storage engine...")
err := ls.Close()
if err != nil {
c.log.Info("storage engine closing failure",
zap.String("error", err.Error()),
)
} else {
c.log.Info("all components of the storage engine closed successfully")
}
})
}
func initObjectPool(cfg *config.Config) (pool cfgObjectRoutines) {
var err error
optNonBlocking := ants.WithNonblocking(true)
pool.putRemoteCapacity = objectconfig.Put(cfg).PoolSizeRemote()
pool.putRemote, err = ants.NewPool(pool.putRemoteCapacity, optNonBlocking)
fatalOnErr(err)
pool.replicatorPoolSize = replicatorconfig.PoolSize(cfg)
if pool.replicatorPoolSize <= 0 {
pool.replicatorPoolSize = pool.putRemoteCapacity
}
pool.replication, err = ants.NewPool(pool.replicatorPoolSize)
fatalOnErr(err)
return pool
}
func (c *cfg) LocalNodeInfo() (*netmapV2.NodeInfo, error) {
var res netmapV2.NodeInfo
ni, ok := c.cfgNetmap.state.getNodeInfo()
if ok {
ni.WriteToV2(&res)
} else {
c.cfgNodeInfo.localInfo.WriteToV2(&res)
}
return &res, nil
}
// handleLocalNodeInfo rewrites local node info from the NeoFS network map.
// Called with nil when storage node is outside the NeoFS network map
// (before entering the network and after leaving it).
func (c *cfg) handleLocalNodeInfo(ni *netmap.NodeInfo) {
c.cfgNetmap.state.setNodeInfo(ni)
}
// bootstrapWithState calls "addPeer" method of the Sidechain Netmap contract
// with the binary-encoded information from the current node's configuration.
// The state is set using the provided setter which MUST NOT be nil.
func (c *cfg) bootstrapWithState(stateSetter func(*netmap.NodeInfo)) error {
ni := c.cfgNodeInfo.localInfo
stateSetter(&ni)
prm := nmClient.AddPeerPrm{}
prm.SetNodeInfo(ni)
return c.cfgNetmap.wrapper.AddPeer(prm)
}
// bootstrapOnline calls cfg.bootstrapWithState with "online" state.
func bootstrapOnline(c *cfg) error {
return c.bootstrapWithState((*netmap.NodeInfo).SetOnline)
}
// bootstrap calls bootstrapWithState with:
// - "maintenance" state if maintenance is in progress on the current node
// - "online", otherwise
func (c *cfg) bootstrap() error {
// switch to online except when under maintenance
st := c.cfgNetmap.state.controlNetmapStatus()
if st == control.NetmapStatus_MAINTENANCE {
c.log.Info("bootstrapping with the maintenance state")
return c.bootstrapWithState((*netmap.NodeInfo).SetMaintenance)
}
c.log.Info("bootstrapping with online state",
zap.Stringer("previous", st),
)
return bootstrapOnline(c)
}
// needBootstrap checks if local node should be registered in network on bootup.
func (c *cfg) needBootstrap() bool {
return c.cfgNetmap.needBootstrap
}
// ObjectServiceLoad implements system loader interface for policer component.
// It is calculated as size/capacity ratio of "remote object put" worker.
// Returns float value between 0.0 and 1.0.
func (c *cfg) ObjectServiceLoad() float64 {
return float64(c.cfgObject.pool.putRemote.Running()) / float64(c.cfgObject.pool.putRemoteCapacity)
}
type dCfg struct {
name string
cfg interface {
Reload() error
}
}
func (c *cfg) configWatcher(ctx context.Context) {
ch := make(chan os.Signal, 1)
signal.Notify(ch, syscall.SIGHUP)
for {
select {
case <-ch:
c.log.Info("SIGHUP has been received, rereading configuration...")
err := c.readConfig(c.appCfg)
if err != nil {
c.log.Error("configuration reading", zap.Error(err))
continue
}
// all the components are expected to support
// Logger's dynamic reconfiguration approach
var components []dCfg
// Logger
logPrm, err := c.loggerPrm()
if err != nil {
c.log.Error("logger configuration preparation", zap.Error(err))
continue
}
components = append(components, dCfg{name: "logger", cfg: logPrm})
// Storage Engine
var rcfg engine.ReConfiguration
for _, optsWithID := range c.shardOpts() {
rcfg.AddShard(optsWithID.configID, optsWithID.shOpts)
}
err = c.cfgObject.cfgLocalStorage.localStorage.Reload(rcfg)
if err != nil {
c.log.Error("storage engine configuration update", zap.Error(err))
continue
}
for _, component := range components {
err = component.cfg.Reload()
if err != nil {
c.log.Error("updated configuration applying",
zap.String("component", component.name),
zap.Error(err))
}
}
c.log.Info("configuration has been reloaded successfully")
case <-ctx.Done():
return
}
}
}

View file

@ -0,0 +1,51 @@
package apiclientconfig
import (
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
const (
subsection = "apiclient"
// DialTimeoutDefault is a default dial timeout of NeoFS API client connection.
DialTimeoutDefault = 5 * time.Second
// StreamTimeoutDefault is a default timeout of NeoFS API streaming operation.
StreamTimeoutDefault = 15 * time.Second
)
// DialTimeout returns the value of "dial_timeout" config parameter
// from "apiclient" section.
//
// Returns DialTimeoutDefault if the value is not positive duration.
func DialTimeout(c *config.Config) time.Duration {
v := config.DurationSafe(c.Sub(subsection), "dial_timeout")
if v > 0 {
return v
}
return DialTimeoutDefault
}
// StreamTimeout returns the value of "stream_timeout" config parameter
// from "apiclient" section.
//
// Returns DialTimeoutDefault if the value is not positive duration.
func StreamTimeout(c *config.Config) time.Duration {
v := config.DurationSafe(c.Sub(subsection), "stream_timeout")
if v > 0 {
return v
}
return StreamTimeoutDefault
}
// AllowExternal returns the value of "allow_external" config parameter
// from "apiclient" section.
//
// Returns false if the value is missing or invalid.
func AllowExternal(c *config.Config) bool {
return config.BoolSafe(c.Sub(subsection), "allow_external")
}

View file

@ -0,0 +1,35 @@
package apiclientconfig_test
import (
"testing"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
apiclientconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/apiclient"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/stretchr/testify/require"
)
func TestApiclientSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
empty := configtest.EmptyConfig()
require.Equal(t, apiclientconfig.DialTimeoutDefault, apiclientconfig.DialTimeout(empty))
require.Equal(t, apiclientconfig.StreamTimeoutDefault, apiclientconfig.StreamTimeout(empty))
require.False(t, apiclientconfig.AllowExternal(empty))
})
const path = "../../../../config/example/node"
var fileConfigTest = func(c *config.Config) {
require.Equal(t, 15*time.Second, apiclientconfig.DialTimeout(c))
require.Equal(t, 20*time.Second, apiclientconfig.StreamTimeout(c))
require.True(t, apiclientconfig.AllowExternal(c))
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,55 @@
package config
import (
"strings"
)
// Sub returns a subsection of the Config by name.
//
// Returns nil if subsection is missing.
func (x *Config) Sub(name string) *Config {
// copy path in order to prevent consequent violations
ln := len(x.path)
path := make([]string, ln, ln+1)
copy(path, x.path)
var defaultPath []string
if x.defaultPath != nil {
ln := len(x.defaultPath)
defaultPath = make([]string, ln, ln+1)
copy(defaultPath, x.defaultPath)
}
return &Config{
v: x.v,
path: append(path, name),
defaultPath: append(defaultPath, name),
}
}
// Value returns the configuration value by name.
//
// Result can be casted to a particular type
// via corresponding function (e.g. StringSlice).
// Note: casting via Go `.()` operator is not
// recommended.
//
// Returns nil if config is nil.
func (x *Config) Value(name string) interface{} {
value := x.v.Get(strings.Join(append(x.path, name), separator))
if value != nil || x.defaultPath == nil {
return value
}
return x.v.Get(strings.Join(append(x.defaultPath, name), separator))
}
// SetDefault sets fallback config for missing values.
//
// It supports only one level of nesting and is intended to be used
// to provide default values.
func (x *Config) SetDefault(from *Config) {
x.defaultPath = make([]string, len(from.path))
copy(x.defaultPath, from.path)
}

View file

@ -0,0 +1,79 @@
package config_test
import (
"os"
"testing"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/internal"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/stretchr/testify/require"
)
func TestConfigCommon(t *testing.T) {
configtest.ForEachFileType("test/config", func(c *config.Config) {
val := c.Value("value")
require.NotNil(t, val)
val = c.Value("non-existent value")
require.Nil(t, val)
sub := c.Sub("section")
require.NotNil(t, sub)
const nonExistentSub = "non-existent sub-section"
val = c.Sub(nonExistentSub).Value("value")
require.Nil(t, val)
})
}
func TestConfigEnv(t *testing.T) {
const (
name = "name"
section = "section"
value = "some value"
)
err := os.Setenv(internal.Env(section, name), value)
require.NoError(t, err)
c := configtest.EmptyConfig()
require.Equal(t, value, c.Sub(section).Value(name))
}
func TestConfig_SubValue(t *testing.T) {
configtest.ForEachFileType("test/config", func(c *config.Config) {
c = c.
Sub("section").
Sub("sub").
Sub("sub")
// get subsection 1
sub := c.Sub("sub1")
// get subsection 2
c.Sub("sub2")
// sub should not be corrupted
require.Equal(t, "val1", sub.Value("key"))
})
}
func TestConfig_SetDefault(t *testing.T) {
configtest.ForEachFileType("test/config", func(c *config.Config) {
c = c.Sub("with_default")
s := c.Sub("custom")
s.SetDefault(c.Sub("default"))
require.Equal(t, int64(42), config.Int(s, "missing"))
require.Equal(t, "b", config.String(s, "overridden"))
require.Equal(t, false, config.Bool(s, "overridden_with_default"))
// Default can be set only once.
s = s.Sub("sub")
require.Equal(t, int64(123), config.Int(s, "missing"))
require.Equal(t, "y", config.String(s, "overridden"))
})
}

View file

@ -0,0 +1,225 @@
package config
import (
"math/bits"
"strings"
"time"
"unicode"
"github.com/spf13/cast"
)
func panicOnErr(err error) {
if err != nil {
panic(err)
}
}
// StringSlice reads a configuration value
// from c by name and casts it to a []string.
//
// Panics if the value can not be casted.
func StringSlice(c *Config, name string) []string {
x, err := cast.ToStringSliceE(c.Value(name))
panicOnErr(err)
return x
}
// StringSliceSafe reads a configuration value
// from c by name and casts it to a []string.
//
// Returns nil if the value can not be casted.
func StringSliceSafe(c *Config, name string) []string {
return cast.ToStringSlice(c.Value(name))
}
// String reads a configuration value
// from c by name and casts it to a string.
//
// Panics if the value can not be casted.
func String(c *Config, name string) string {
x, err := cast.ToStringE(c.Value(name))
panicOnErr(err)
return x
}
// StringSafe reads a configuration value
// from c by name and casts it to a string.
//
// Returns "" if the value can not be casted.
func StringSafe(c *Config, name string) string {
return cast.ToString(c.Value(name))
}
// Duration reads a configuration value
// from c by name and casts it to time.Duration.
//
// Panics if the value can not be casted.
func Duration(c *Config, name string) time.Duration {
x, err := cast.ToDurationE(c.Value(name))
panicOnErr(err)
return x
}
// DurationSafe reads a configuration value
// from c by name and casts it to time.Duration.
//
// Returns 0 if the value can not be casted.
func DurationSafe(c *Config, name string) time.Duration {
return cast.ToDuration(c.Value(name))
}
// Bool reads a configuration value
// from c by name and casts it to bool.
//
// Panics if the value can not be casted.
func Bool(c *Config, name string) bool {
x, err := cast.ToBoolE(c.Value(name))
panicOnErr(err)
return x
}
// BoolSafe reads a configuration value
// from c by name and casts it to bool.
//
// Returns false if the value can not be casted.
func BoolSafe(c *Config, name string) bool {
return cast.ToBool(c.Value(name))
}
// Uint32 reads a configuration value
// from c by name and casts it to uint32.
//
// Panics if the value can not be casted.
func Uint32(c *Config, name string) uint32 {
x, err := cast.ToUint32E(c.Value(name))
panicOnErr(err)
return x
}
// Uint32Safe reads a configuration value
// from c by name and casts it to uint32.
//
// Returns 0 if the value can not be casted.
func Uint32Safe(c *Config, name string) uint32 {
return cast.ToUint32(c.Value(name))
}
// Uint reads a configuration value
// from c by name and casts it to uint64.
//
// Panics if the value can not be casted.
func Uint(c *Config, name string) uint64 {
x, err := cast.ToUint64E(c.Value(name))
panicOnErr(err)
return x
}
// UintSafe reads a configuration value
// from c by name and casts it to uint64.
//
// Returns 0 if the value can not be casted.
func UintSafe(c *Config, name string) uint64 {
return cast.ToUint64(c.Value(name))
}
// Int reads a configuration value
// from c by name and casts it to int64.
//
// Panics if the value can not be casted.
func Int(c *Config, name string) int64 {
x, err := cast.ToInt64E(c.Value(name))
panicOnErr(err)
return x
}
// IntSafe reads a configuration value
// from c by name and casts it to int64.
//
// Returns 0 if the value can not be casted.
func IntSafe(c *Config, name string) int64 {
return cast.ToInt64(c.Value(name))
}
// SizeInBytesSafe reads a configuration value
// from c by name and casts it to size in bytes (uint64).
//
// The suffix can be single-letter (b, k, m, g, t) or with
// an additional b at the end. Spaces between the number and the suffix
// are allowed. All multipliers are power of 2 (i.e. k is for kibi-byte).
//
// Returns 0 if a value can't be casted.
func SizeInBytesSafe(c *Config, name string) uint64 {
s := StringSafe(c, name)
return parseSizeInBytes(s)
}
// The following code is taken from https://github.com/spf13/viper/blob/master/util.go
// with minor corrections (allow to use both `k` and `kb` forms.
// Seems like viper allows to convert sizes but corresponding parser in `cast` package
// is missing.
// safeMul returns size*multiplier, rounding down to the
// multiplier/1024 number of bytes.
// Returns 0 if overflow is detected.
func safeMul(size float64, multiplier uint64) uint64 {
n := uint64(size)
f := uint64((size - float64(n)) * 1024)
if f != 0 && multiplier != 1 {
s := n<<10 + f
if s < n {
return 0
}
n = s
multiplier >>= 10
}
hi, lo := bits.Mul64(n, multiplier)
if hi != 0 {
return 0
}
return lo
}
// parseSizeInBytes converts strings like 1GB or 12 mb into an unsigned integer number of bytes.
func parseSizeInBytes(sizeStr string) uint64 {
sizeStr = strings.TrimSpace(sizeStr)
lastChar := len(sizeStr) - 1
multiplier := uint64(1)
if lastChar > 0 {
if sizeStr[lastChar] == 'b' || sizeStr[lastChar] == 'B' {
lastChar--
}
if lastChar > 0 {
switch unicode.ToLower(rune(sizeStr[lastChar])) {
case 'k':
multiplier = 1 << 10
sizeStr = strings.TrimSpace(sizeStr[:lastChar])
case 'm':
multiplier = 1 << 20
sizeStr = strings.TrimSpace(sizeStr[:lastChar])
case 'g':
multiplier = 1 << 30
sizeStr = strings.TrimSpace(sizeStr[:lastChar])
case 't':
multiplier = 1 << 40
sizeStr = strings.TrimSpace(sizeStr[:lastChar])
default:
multiplier = 1
sizeStr = strings.TrimSpace(sizeStr[:lastChar+1])
}
}
}
size := cast.ToFloat64(sizeStr)
return safeMul(size, multiplier)
}

View file

@ -0,0 +1,139 @@
package config_test
import (
"testing"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/stretchr/testify/require"
)
func TestStringSlice(t *testing.T) {
configtest.ForEachFileType("test/config", func(c *config.Config) {
cStringSlice := c.Sub("string_slice")
val := config.StringSlice(cStringSlice, "empty")
require.Empty(t, val)
val = config.StringSlice(cStringSlice, "filled")
require.Equal(t, []string{
"string1",
"string2",
}, val)
require.Panics(t, func() {
config.StringSlice(cStringSlice, "incorrect")
})
val = config.StringSliceSafe(cStringSlice, "incorrect")
require.Nil(t, val)
})
}
func TestString(t *testing.T) {
configtest.ForEachFileType("test/config", func(c *config.Config) {
c = c.Sub("string")
val := config.String(c, "correct")
require.Equal(t, "some string", val)
require.Panics(t, func() {
config.String(c, "incorrect")
})
val = config.StringSafe(c, "incorrect")
require.Empty(t, val)
})
}
func TestDuration(t *testing.T) {
configtest.ForEachFileType("test/config", func(c *config.Config) {
c = c.Sub("duration")
val := config.Duration(c, "correct")
require.Equal(t, 15*time.Minute, val)
require.Panics(t, func() {
config.Duration(c, "incorrect")
})
val = config.DurationSafe(c, "incorrect")
require.Equal(t, time.Duration(0), val)
})
}
func TestBool(t *testing.T) {
configtest.ForEachFileType("test/config", func(c *config.Config) {
c = c.Sub("bool")
val := config.Bool(c, "correct")
require.Equal(t, true, val)
val = config.Bool(c, "correct_string")
require.Equal(t, true, val)
require.Panics(t, func() {
config.Bool(c, "incorrect")
})
val = config.BoolSafe(c, "incorrect")
require.Equal(t, false, val)
})
}
func TestNumbers(t *testing.T) {
configtest.ForEachFileType("test/config", func(c *config.Config) {
c = c.Sub("number")
const (
intPos = "int_pos"
intNeg = "int_neg"
fractPos = "fract_pos"
fractNeg = "fract_neg"
incorrect = "incorrect"
)
require.EqualValues(t, 1, config.Int(c, intPos))
require.EqualValues(t, 1, config.Uint(c, intPos))
require.EqualValues(t, -1, config.Int(c, intNeg))
require.Panics(t, func() { config.Uint(c, intNeg) })
require.EqualValues(t, 2, config.Int(c, fractPos))
require.EqualValues(t, 2, config.Uint(c, fractPos))
require.EqualValues(t, -2, config.Int(c, fractNeg))
require.Panics(t, func() { config.Uint(c, fractNeg) })
require.Panics(t, func() { config.Int(c, incorrect) })
require.Panics(t, func() { config.Uint(c, incorrect) })
require.Zero(t, config.IntSafe(c, incorrect))
require.Zero(t, config.UintSafe(c, incorrect))
})
}
func TestSizeInBytes(t *testing.T) {
const (
kb = 1024
mb = 1024 * kb
gb = 1024 * mb
tb = 1024 * gb
)
configtest.ForEachFileType("test/config", func(c *config.Config) {
c = c.Sub("sizes")
require.EqualValues(t, kb, config.SizeInBytesSafe(c, "size_kb"))
require.EqualValues(t, 2*kb, config.SizeInBytesSafe(c, "size_kb_no_space"))
require.EqualValues(t, 12*mb, config.SizeInBytesSafe(c, "size_mb"))
require.EqualValues(t, 4*gb, config.SizeInBytesSafe(c, "size_gb"))
require.EqualValues(t, 5*tb, config.SizeInBytesSafe(c, "size_tb"))
require.EqualValues(t, 12, config.SizeInBytesSafe(c, "size_i_am_not_very_clever"))
require.EqualValues(t, tb/2, config.SizeInBytesSafe(c, "size_float"))
require.EqualValues(t, uint64(14*gb+(gb*123/1000/mb*mb)), config.SizeInBytesSafe(c, "size_float_big"))
require.EqualValues(t, 2048, config.SizeInBytesSafe(c, "size_bytes"))
require.EqualValues(t, 123456, config.SizeInBytesSafe(c, "size_bytes_no_suffix"))
})
}

View file

@ -0,0 +1,73 @@
package config
import (
"fmt"
"strings"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/internal"
"github.com/spf13/viper"
)
// Config represents a group of named values structured
// by tree type.
//
// Sub-trees are named configuration sub-sections,
// leaves are named configuration values.
// Names are of string type.
type Config struct {
v *viper.Viper
opts opts
defaultPath []string
path []string
}
const separator = "."
// Prm groups required parameters of the Config.
type Prm struct{}
// New creates a new Config instance.
//
// If file option is provided (WithConfigFile),
// configuration values are read from it.
// Otherwise, Config is a degenerate tree.
func New(_ Prm, opts ...Option) *Config {
v := viper.New()
v.SetEnvPrefix(internal.EnvPrefix)
v.AutomaticEnv()
v.SetEnvKeyReplacer(strings.NewReplacer(separator, internal.EnvSeparator))
o := defaultOpts()
for i := range opts {
opts[i](o)
}
if o.path != "" {
v.SetConfigFile(o.path)
err := v.ReadInConfig()
if err != nil {
panic(fmt.Errorf("failed to read config: %w", err))
}
}
return &Config{
v: v,
opts: *o,
}
}
// Reload reads configuration path if it was provided to New.
func (x *Config) Reload() error {
if x.opts.path != "" {
err := x.v.ReadInConfig()
if err != nil {
return fmt.Errorf("rereading configuration file: %w", err)
}
}
return nil
}

View file

@ -0,0 +1,76 @@
package contractsconfig
import (
"fmt"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
"github.com/nspcc-dev/neo-go/pkg/util"
)
const (
subsection = "contracts"
)
// Netmap returns the value of "netmap" config parameter
// from "contracts" section.
//
// Returns zero filled script hash if the value is not set.
// Throws panic if the value is not a 20-byte LE hex-encoded string.
func Netmap(c *config.Config) util.Uint160 {
return contractAddress(c, "netmap")
}
// Balance returns the value of "balance" config parameter
// from "contracts" section.
//
// Returns zero filled script hash if the value is not set.
// Throws panic if the value is not a 20-byte LE hex-encoded string.
func Balance(c *config.Config) util.Uint160 {
return contractAddress(c, "balance")
}
// Container returns the value of "container" config parameter
// from "contracts" section.
//
// Returns zero filled script hash if the value is not set.
// Throws panic if the value is not a 20-byte LE hex-encoded string.
func Container(c *config.Config) util.Uint160 {
return contractAddress(c, "container")
}
// Reputation returnsthe value of "reputation" config parameter
// from "contracts" section.
//
// Returns zero filled script hash if the value is not set.
// Throws panic if the value is not a 20-byte LE hex-encoded string.
func Reputation(c *config.Config) util.Uint160 {
return contractAddress(c, "reputation")
}
// Proxy returnsthe value of "proxy" config parameter
// from "contracts" section.
//
// Returns zero filled script hash if the value is not set.
// Throws panic if the value is not a 20-byte LE hex-encoded string.
func Proxy(c *config.Config) util.Uint160 {
return contractAddress(c, "proxy")
}
func contractAddress(c *config.Config, name string) util.Uint160 {
v := config.String(c.Sub(subsection), name)
if v == "" {
return util.Uint160{} // if address is not set, then NNS resolver should be used
}
addr, err := util.Uint160DecodeStringLE(v)
if err != nil {
panic(fmt.Errorf(
"can't parse %s contract address %s: %w",
name,
v,
err,
))
}
return addr
}

View file

@ -0,0 +1,61 @@
package contractsconfig_test
import (
"testing"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
contractsconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/contracts"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/nspcc-dev/neo-go/pkg/util"
"github.com/stretchr/testify/require"
)
func TestContractsSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
empty := configtest.EmptyConfig()
emptyHash := util.Uint160{}
require.Equal(t, emptyHash, contractsconfig.Balance(empty))
require.Equal(t, emptyHash, contractsconfig.Container(empty))
require.Equal(t, emptyHash, contractsconfig.Netmap(empty))
require.Equal(t, emptyHash, contractsconfig.Reputation(empty))
require.Equal(t, emptyHash, contractsconfig.Proxy(empty))
})
const path = "../../../../config/example/node"
expBalance, err := util.Uint160DecodeStringLE("5263abba1abedbf79bb57f3e40b50b4425d2d6cd")
require.NoError(t, err)
expConatiner, err := util.Uint160DecodeStringLE("5d084790d7aa36cea7b53fe897380dab11d2cd3c")
require.NoError(t, err)
expNetmap, err := util.Uint160DecodeStringLE("0cce9e948dca43a6b592efe59ddb4ecb89bdd9ca")
require.NoError(t, err)
expReputation, err := util.Uint160DecodeStringLE("441995f631c1da2b133462b71859494a5cd45e90")
require.NoError(t, err)
expProxy, err := util.Uint160DecodeStringLE("ad7c6b55b737b696e5c82c85445040964a03e97f")
require.NoError(t, err)
var fileConfigTest = func(c *config.Config) {
balance := contractsconfig.Balance(c)
container := contractsconfig.Container(c)
netmap := contractsconfig.Netmap(c)
reputation := contractsconfig.Reputation(c)
proxy := contractsconfig.Proxy(c)
require.Equal(t, expBalance, balance)
require.Equal(t, expConatiner, container)
require.Equal(t, expNetmap, netmap)
require.Equal(t, expReputation, reputation)
require.Equal(t, expProxy, proxy)
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,62 @@
package controlconfig
import (
"fmt"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
)
// GRPCConfig is a wrapper over "grpc" config section which provides access
// to gRPC configuration of control service.
type GRPCConfig struct {
cfg *config.Config
}
const (
subsection = "control"
grpcSubsection = "grpc"
// GRPCEndpointDefault is a default endpoint of gRPC Control service.
GRPCEndpointDefault = ""
)
// AuthorizedKeys parses and returns an array of "authorized_keys" config
// parameter from "control" section.
//
// Returns an empty list if not set.
func AuthorizedKeys(c *config.Config) keys.PublicKeys {
strKeys := config.StringSliceSafe(c.Sub(subsection), "authorized_keys")
pubs := make(keys.PublicKeys, 0, len(strKeys))
for i := range strKeys {
pub, err := keys.NewPublicKeyFromString(strKeys[i])
if err != nil {
panic(fmt.Errorf("invalid permitted key for Control service %s: %w", strKeys[i], err))
}
pubs = append(pubs, pub)
}
return pubs
}
// GRPC returns a structure that provides access to "grpc" subsection of
// "control" section.
func GRPC(c *config.Config) GRPCConfig {
return GRPCConfig{
c.Sub(subsection).Sub(grpcSubsection),
}
}
// Endpoint returns the value of "endpoint" config parameter.
//
// Returns GRPCEndpointDefault if the value is not a non-empty string.
func (g GRPCConfig) Endpoint() string {
v := config.String(g.cfg, "endpoint")
if v != "" {
return v
}
return GRPCEndpointDefault
}

View file

@ -0,0 +1,37 @@
package controlconfig_test
import (
"testing"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
controlconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/control"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
"github.com/stretchr/testify/require"
)
func TestControlSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
empty := configtest.EmptyConfig()
require.Empty(t, controlconfig.AuthorizedKeys(empty))
require.Equal(t, controlconfig.GRPCEndpointDefault, controlconfig.GRPC(empty).Endpoint())
})
const path = "../../../../config/example/node"
pubs := make(keys.PublicKeys, 2)
pubs[0], _ = keys.NewPublicKeyFromString("035839e45d472a3b7769a2a1bd7d54c4ccd4943c3b40f547870e83a8fcbfb3ce11")
pubs[1], _ = keys.NewPublicKeyFromString("028f42cfcb74499d7b15b35d9bff260a1c8d27de4f446a627406a382d8961486d6")
var fileConfigTest = func(c *config.Config) {
require.Equal(t, pubs, controlconfig.AuthorizedKeys(c))
require.Equal(t, "localhost:8090", controlconfig.GRPC(c).Endpoint())
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,85 @@
package engineconfig
import (
"errors"
"strconv"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
shardconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
)
const (
subsection = "storage"
// ShardPoolSizeDefault is a default value of routine pool size per-shard to
// process object PUT operations in a storage engine.
ShardPoolSizeDefault = 20
)
// ErrNoShardConfigured is returned when at least 1 shard is required but none are found.
var ErrNoShardConfigured = errors.New("no shard configured")
// IterateShards iterates over subsections of "shard" subsection of "storage" section of c,
// wrap them into shardconfig.Config and passes to f.
//
// Section names are expected to be consecutive integer numbers, starting from 0.
//
// Panics if N is not a positive number while shards are required.
func IterateShards(c *config.Config, required bool, f func(*shardconfig.Config) error) error {
c = c.Sub(subsection)
c = c.Sub("shard")
def := c.Sub("default")
alive := 0
i := uint64(0)
for ; ; i++ {
si := strconv.FormatUint(i, 10)
sc := shardconfig.From(
c.Sub(si),
)
// Path for the blobstor can't be present in the default section, because different shards
// must have different paths, so if it is missing, the shard is not here.
// At the same time checking for "blobstor" section doesn't work proper
// with configuration via the environment.
if (*config.Config)(sc).Value("metabase.path") == nil {
break
}
(*config.Config)(sc).SetDefault(def)
if sc.Mode() == mode.Disabled {
continue
}
if err := f(sc); err != nil {
return err
}
alive++
}
if alive == 0 && required {
return ErrNoShardConfigured
}
return nil
}
// ShardPoolSize returns the value of "shard_pool_size" config parameter from "storage" section.
//
// Returns ShardPoolSizeDefault if the value is not a positive number.
func ShardPoolSize(c *config.Config) uint32 {
v := config.Uint32Safe(c.Sub(subsection), "shard_pool_size")
if v > 0 {
return v
}
return ShardPoolSizeDefault
}
// ShardErrorThreshold returns the value of "shard_ro_error_threshold" config parameter from "storage" section.
//
// Returns 0 if the the value is missing.
func ShardErrorThreshold(c *config.Config) uint32 {
return config.Uint32Safe(c.Sub(subsection), "shard_ro_error_threshold")
}

View file

@ -0,0 +1,167 @@
package engineconfig_test
import (
"io/fs"
"testing"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
engineconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine"
shardconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard"
blobovniczaconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor/blobovnicza"
fstreeconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor/fstree"
piloramaconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/pilorama"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"github.com/stretchr/testify/require"
)
func TestEngineSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
empty := configtest.EmptyConfig()
require.ErrorIs(t,
engineconfig.IterateShards(empty, true, nil),
engineconfig.ErrNoShardConfigured)
handlerCalled := false
require.NoError(t,
engineconfig.IterateShards(empty, false, func(_ *shardconfig.Config) error {
handlerCalled = true
return nil
}))
require.False(t, handlerCalled)
require.EqualValues(t, 0, engineconfig.ShardErrorThreshold(empty))
require.EqualValues(t, engineconfig.ShardPoolSizeDefault, engineconfig.ShardPoolSize(empty))
require.EqualValues(t, mode.ReadWrite, shardconfig.From(empty).Mode())
})
const path = "../../../../config/example/node"
var fileConfigTest = func(c *config.Config) {
num := 0
require.EqualValues(t, 100, engineconfig.ShardErrorThreshold(c))
require.EqualValues(t, 15, engineconfig.ShardPoolSize(c))
err := engineconfig.IterateShards(c, true, func(sc *shardconfig.Config) error {
defer func() {
num++
}()
wc := sc.WriteCache()
meta := sc.Metabase()
blob := sc.BlobStor()
ss := blob.Storages()
pl := sc.Pilorama()
gc := sc.GC()
switch num {
case 0:
require.Equal(t, "tmp/0/blob/pilorama.db", pl.Path())
require.Equal(t, fs.FileMode(piloramaconfig.PermDefault), pl.Perm())
require.False(t, pl.NoSync())
require.Equal(t, pl.MaxBatchDelay(), 10*time.Millisecond)
require.Equal(t, pl.MaxBatchSize(), 200)
require.Equal(t, false, wc.Enabled())
require.Equal(t, true, wc.NoSync())
require.Equal(t, "tmp/0/cache", wc.Path())
require.EqualValues(t, 16384, wc.SmallObjectSize())
require.EqualValues(t, 134217728, wc.MaxObjectSize())
require.EqualValues(t, 30, wc.WorkersNumber())
require.EqualValues(t, 3221225472, wc.SizeLimit())
require.Equal(t, "tmp/0/meta", meta.Path())
require.Equal(t, fs.FileMode(0644), meta.BoltDB().Perm())
require.Equal(t, 100, meta.BoltDB().MaxBatchSize())
require.Equal(t, 10*time.Millisecond, meta.BoltDB().MaxBatchDelay())
require.Equal(t, true, sc.Compress())
require.Equal(t, []string{"audio/*", "video/*"}, sc.UncompressableContentTypes())
require.EqualValues(t, 102400, sc.SmallSizeLimit())
require.Equal(t, 2, len(ss))
blz := blobovniczaconfig.From((*config.Config)(ss[0]))
require.Equal(t, "tmp/0/blob/blobovnicza", ss[0].Path())
require.EqualValues(t, 0644, blz.BoltDB().Perm())
require.EqualValues(t, 4194304, blz.Size())
require.EqualValues(t, 1, blz.ShallowDepth())
require.EqualValues(t, 4, blz.ShallowWidth())
require.EqualValues(t, 50, blz.OpenedCacheSize())
require.Equal(t, "tmp/0/blob", ss[1].Path())
require.EqualValues(t, 0644, ss[1].Perm())
fst := fstreeconfig.From((*config.Config)(ss[1]))
require.EqualValues(t, 5, fst.Depth())
require.Equal(t, false, fst.NoSync())
require.EqualValues(t, 150, gc.RemoverBatchSize())
require.Equal(t, 2*time.Minute, gc.RemoverSleepInterval())
require.Equal(t, false, sc.RefillMetabase())
require.Equal(t, mode.ReadOnly, sc.Mode())
case 1:
require.Equal(t, "tmp/1/blob/pilorama.db", pl.Path())
require.Equal(t, fs.FileMode(0644), pl.Perm())
require.True(t, pl.NoSync())
require.Equal(t, 5*time.Millisecond, pl.MaxBatchDelay())
require.Equal(t, 100, pl.MaxBatchSize())
require.Equal(t, true, wc.Enabled())
require.Equal(t, false, wc.NoSync())
require.Equal(t, "tmp/1/cache", wc.Path())
require.EqualValues(t, 16384, wc.SmallObjectSize())
require.EqualValues(t, 134217728, wc.MaxObjectSize())
require.EqualValues(t, 30, wc.WorkersNumber())
require.EqualValues(t, 4294967296, wc.SizeLimit())
require.Equal(t, "tmp/1/meta", meta.Path())
require.Equal(t, fs.FileMode(0644), meta.BoltDB().Perm())
require.Equal(t, 200, meta.BoltDB().MaxBatchSize())
require.Equal(t, 20*time.Millisecond, meta.BoltDB().MaxBatchDelay())
require.Equal(t, false, sc.Compress())
require.Equal(t, []string(nil), sc.UncompressableContentTypes())
require.EqualValues(t, 102400, sc.SmallSizeLimit())
require.Equal(t, 2, len(ss))
blz := blobovniczaconfig.From((*config.Config)(ss[0]))
require.Equal(t, "tmp/1/blob/blobovnicza", ss[0].Path())
require.EqualValues(t, 4194304, blz.Size())
require.EqualValues(t, 1, blz.ShallowDepth())
require.EqualValues(t, 4, blz.ShallowWidth())
require.EqualValues(t, 50, blz.OpenedCacheSize())
require.Equal(t, "tmp/1/blob", ss[1].Path())
require.EqualValues(t, 0644, ss[1].Perm())
fst := fstreeconfig.From((*config.Config)(ss[1]))
require.EqualValues(t, 5, fst.Depth())
require.Equal(t, true, fst.NoSync())
require.EqualValues(t, 200, gc.RemoverBatchSize())
require.Equal(t, 5*time.Minute, gc.RemoverSleepInterval())
require.Equal(t, true, sc.RefillMetabase())
require.Equal(t, mode.ReadWrite, sc.Mode())
}
return nil
})
require.NoError(t, err)
require.Equal(t, 2, num)
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,104 @@
package blobovniczaconfig
import (
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
boltdbconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/boltdb"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/blobovniczatree"
)
// Config is a wrapper over the config section
// which provides access to Blobovnicza configurations.
type Config config.Config
const (
// SizeDefault is a default limit of estimates of Blobovnicza size.
SizeDefault = 1 << 30
// ShallowDepthDefault is a default shallow dir depth.
ShallowDepthDefault = 2
// ShallowWidthDefault is a default shallow dir width.
ShallowWidthDefault = 16
// OpenedCacheSizeDefault is a default cache size of opened Blobovnicza's.
OpenedCacheSizeDefault = 16
)
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Type returns the storage type.
func (x *Config) Type() string {
return blobovniczatree.Type
}
// Size returns the value of "size" config parameter.
//
// Returns SizeDefault if the value is not a positive number.
func (x *Config) Size() uint64 {
s := config.SizeInBytesSafe(
(*config.Config)(x),
"size",
)
if s > 0 {
return s
}
return SizeDefault
}
// ShallowDepth returns the value of "depth" config parameter.
//
// Returns ShallowDepthDefault if the value is not a positive number.
func (x *Config) ShallowDepth() uint64 {
d := config.UintSafe(
(*config.Config)(x),
"depth",
)
if d > 0 {
return d
}
return ShallowDepthDefault
}
// ShallowWidth returns the value of "width" config parameter.
//
// Returns ShallowWidthDefault if the value is not a positive number.
func (x *Config) ShallowWidth() uint64 {
d := config.UintSafe(
(*config.Config)(x),
"width",
)
if d > 0 {
return d
}
return ShallowWidthDefault
}
// OpenedCacheSize returns the value of "opened_cache_capacity" config parameter.
//
// Returns OpenedCacheSizeDefault if the value is not a positive number.
func (x *Config) OpenedCacheSize() int {
d := config.IntSafe(
(*config.Config)(x),
"opened_cache_capacity",
)
if d > 0 {
return int(d)
}
return OpenedCacheSizeDefault
}
// BoltDB returns config instance for querying bolt db specific parameters.
func (x *Config) BoltDB() *boltdbconfig.Config {
return (*boltdbconfig.Config)(x)
}

View file

@ -0,0 +1,38 @@
package blobstorconfig
import (
"strconv"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor/storage"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/blobovniczatree"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/fstree"
)
// Config is a wrapper over the config section
// which provides access to BlobStor configurations.
type Config config.Config
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Storages returns the value of storage subcomponents.
func (x *Config) Storages() []*storage.Config {
var ss []*storage.Config
for i := 0; ; i++ {
typ := config.String(
(*config.Config)(x),
strconv.Itoa(i)+".type")
switch typ {
case "":
return ss
case fstree.Type, blobovniczatree.Type:
sub := storage.From((*config.Config)(x).Sub(strconv.Itoa(i)))
ss = append(ss, sub)
default:
panic("invalid type")
}
}
}

View file

@ -0,0 +1,47 @@
package fstree
import (
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/fstree"
)
// Config is a wrapper over the config section
// which provides access to Blobovnicza configurations.
type Config config.Config
// DepthDefault is a default shallow dir depth.
const DepthDefault = 4
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Type returns the storage type.
func (x *Config) Type() string {
return fstree.Type
}
// Depth returns the value of "depth" config parameter.
//
// Returns DepthDefault if the value is out of
// [1:fstree.MaxDepth] range.
func (x *Config) Depth() uint64 {
d := config.UintSafe(
(*config.Config)(x),
"depth",
)
if d >= 1 && d <= fstree.MaxDepth {
return d
}
return DepthDefault
}
// NoSync returns the value of "no_sync" config parameter.
//
// Returns false if the value is not a boolean or is missing.
func (x *Config) NoSync() bool {
return config.BoolSafe((*config.Config)(x), "no_sync")
}

View file

@ -0,0 +1,55 @@
package storage
import (
"io/fs"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
type Config config.Config
// PermDefault are default permission bits for BlobStor data.
const PermDefault = 0660
func From(x *config.Config) *Config {
return (*Config)(x)
}
// Type returns storage type.
func (x *Config) Type() string {
return config.String(
(*config.Config)(x),
"type")
}
// Path returns the value of "path" config parameter.
//
// Panics if the value is not a non-empty string.
func (x *Config) Path() string {
p := config.String(
(*config.Config)(x),
"path",
)
if p == "" {
panic("blobstor path not set")
}
return p
}
// Perm returns the value of "perm" config parameter as a fs.FileMode.
//
// Returns PermDefault if the value is not a non-zero number.
func (x *Config) Perm() fs.FileMode {
p := config.UintSafe(
(*config.Config)(x),
"perm",
)
if p == 0 {
p = PermDefault
}
return fs.FileMode(p)
}

View file

@ -0,0 +1,62 @@
package boltdbconfig
import (
"io/fs"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
// Config is a wrapper over the config section
// which provides access to boltdb specific parameters.
type Config config.Config
const (
// PermDefault is a default permission bits for metabase file.
PermDefault = 0660
)
// Perm returns the value of "perm" config parameter as a fs.FileMode.
//
// Returns PermDefault if the value is not a positive number.
func (x *Config) Perm() fs.FileMode {
p := config.UintSafe(
(*config.Config)(x),
"perm",
)
if p == 0 {
p = PermDefault
}
return fs.FileMode(p)
}
// MaxBatchDelay returns the value of "max_batch_delay" config parameter.
//
// Returns 0 if the value is not a positive number.
func (x *Config) MaxBatchDelay() time.Duration {
d := config.DurationSafe((*config.Config)(x), "max_batch_delay")
if d < 0 {
d = 0
}
return d
}
// MaxBatchSize returns the value of "max_batch_size" config parameter.
//
// Returns 0 if the value is not a positive number.
func (x *Config) MaxBatchSize() int {
s := int(config.IntSafe((*config.Config)(x), "max_batch_size"))
if s < 0 {
s = 0
}
return s
}
// NoSync returns the value of "no_sync" config parameter.
//
// Returns false if the value is not a boolean.
func (x *Config) NoSync() bool {
return config.BoolSafe((*config.Config)(x), "no_sync")
}

View file

@ -0,0 +1,138 @@
package shardconfig
import (
"fmt"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
blobstorconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor"
gcconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/gc"
metabaseconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/metabase"
piloramaconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/pilorama"
writecacheconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/writecache"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
)
// Config is a wrapper over the config section
// which provides access to Shard configurations.
type Config config.Config
// SmallSizeLimitDefault is a default limit of small objects payload in bytes.
const SmallSizeLimitDefault = 1 << 20
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Compress returns the value of "compress" config parameter.
//
// Returns false if the value is not a valid bool.
func (x *Config) Compress() bool {
return config.BoolSafe(
(*config.Config)(x),
"compress",
)
}
// UncompressableContentTypes returns the value of "compress_skip_content_types" config parameter.
//
// Returns nil if a the value is missing or is invalid.
func (x *Config) UncompressableContentTypes() []string {
return config.StringSliceSafe(
(*config.Config)(x),
"compression_exclude_content_types")
}
// SmallSizeLimit returns the value of "small_object_size" config parameter.
//
// Returns SmallSizeLimitDefault if the value is not a positive number.
func (x *Config) SmallSizeLimit() uint64 {
l := config.SizeInBytesSafe(
(*config.Config)(x),
"small_object_size",
)
if l > 0 {
return l
}
return SmallSizeLimitDefault
}
// BlobStor returns "blobstor" subsection as a blobstorconfig.Config.
func (x *Config) BlobStor() *blobstorconfig.Config {
return blobstorconfig.From(
(*config.Config)(x).
Sub("blobstor"),
)
}
// Metabase returns "metabase" subsection as a metabaseconfig.Config.
func (x *Config) Metabase() *metabaseconfig.Config {
return metabaseconfig.From(
(*config.Config)(x).
Sub("metabase"),
)
}
// WriteCache returns "writecache" subsection as a writecacheconfig.Config.
func (x *Config) WriteCache() *writecacheconfig.Config {
return writecacheconfig.From(
(*config.Config)(x).
Sub("writecache"),
)
}
// Pilorama returns "pilorama" subsection as a piloramaconfig.Config.
func (x *Config) Pilorama() *piloramaconfig.Config {
return piloramaconfig.From(
(*config.Config)(x).
Sub("pilorama"),
)
}
// GC returns "gc" subsection as a gcconfig.Config.
func (x *Config) GC() *gcconfig.Config {
return gcconfig.From(
(*config.Config)(x).
Sub("gc"),
)
}
// RefillMetabase returns the value of "resync_metabase" config parameter.
//
// Returns false if the value is not a valid bool.
func (x *Config) RefillMetabase() bool {
return config.BoolSafe(
(*config.Config)(x),
"resync_metabase",
)
}
// Mode return the value of "mode" config parameter.
//
// Panics if read the value is not one of predefined
// shard modes.
func (x *Config) Mode() (m mode.Mode) {
s := config.StringSafe(
(*config.Config)(x),
"mode",
)
switch s {
case "read-write", "":
m = mode.ReadWrite
case "read-only":
m = mode.ReadOnly
case "degraded":
m = mode.Degraded
case "degraded-read-only":
m = mode.DegradedReadOnly
case "disabled":
m = mode.Disabled
default:
panic(fmt.Sprintf("unknown shard mode: %s", s))
}
return
}

View file

@ -0,0 +1,58 @@
package gcconfig
import (
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
// Config is a wrapper over the config section
// which provides access to Shard's GC configurations.
type Config config.Config
const (
// RemoverBatchSizeDefault is a default batch size for Shard GC's remover.
RemoverBatchSizeDefault = 100
// RemoverSleepIntervalDefault is a default sleep interval of Shard GC's remover.
RemoverSleepIntervalDefault = time.Minute
)
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// RemoverBatchSize returns the value of "remover_batch_size"
// config parameter.
//
// Returns RemoverBatchSizeDefault if the value is not a positive number.
func (x *Config) RemoverBatchSize() int {
s := config.IntSafe(
(*config.Config)(x),
"remover_batch_size",
)
if s > 0 {
return int(s)
}
return RemoverBatchSizeDefault
}
// RemoverSleepInterval returns the value of "remover_sleep_interval"
// config parameter.
//
// Returns RemoverSleepIntervalDefault if the value is not a positive number.
func (x *Config) RemoverSleepInterval() time.Duration {
s := config.DurationSafe(
(*config.Config)(x),
"remover_sleep_interval",
)
if s > 0 {
return s
}
return RemoverSleepIntervalDefault
}

View file

@ -0,0 +1,36 @@
package metabaseconfig
import (
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
boltdbconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/boltdb"
)
// Config is a wrapper over the config section
// which provides access to Metabase configurations.
type Config config.Config
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Path returns the value of "path" config parameter.
//
// Panics if the value is not a non-empty string.
func (x *Config) Path() string {
p := config.String(
(*config.Config)(x),
"path",
)
if p == "" {
panic("metabase path not set")
}
return p
}
// BoltDB returns config instance for querying bolt db specific parameters.
func (x *Config) BoltDB() *boltdbconfig.Config {
return (*boltdbconfig.Config)(x)
}

View file

@ -0,0 +1,70 @@
package piloramaconfig
import (
"io/fs"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
// Config is a wrapper over the config section
// which provides access to Metabase configurations.
type Config config.Config
const (
// PermDefault is a default permission bits for metabase file.
PermDefault = 0660
)
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Path returns the value of "path" config parameter.
//
// Returns empty string if missing, for compatibility with older configurations.
func (x *Config) Path() string {
return config.String((*config.Config)(x), "path")
}
// Perm returns the value of "perm" config parameter as a fs.FileMode.
//
// Returns PermDefault if the value is not a positive number.
func (x *Config) Perm() fs.FileMode {
p := config.UintSafe((*config.Config)(x), "perm")
if p == 0 {
p = PermDefault
}
return fs.FileMode(p)
}
// NoSync returns the value of "no_sync" config parameter as a bool value.
//
// Returns false if the value is not a boolean.
func (x *Config) NoSync() bool {
return config.BoolSafe((*config.Config)(x), "no_sync")
}
// MaxBatchDelay returns the value of "max_batch_delay" config parameter.
//
// Returns 0 if the value is not a positive number.
func (x *Config) MaxBatchDelay() time.Duration {
d := config.DurationSafe((*config.Config)(x), "max_batch_delay")
if d <= 0 {
d = 0
}
return d
}
// MaxBatchSize returns the value of "max_batch_size" config parameter.
//
// Returns 0 if the value is not a positive number.
func (x *Config) MaxBatchSize() int {
s := int(config.IntSafe((*config.Config)(x), "max_batch_size"))
if s <= 0 {
s = 0
}
return s
}

View file

@ -0,0 +1,128 @@
package writecacheconfig
import (
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
boltdbconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/boltdb"
)
// Config is a wrapper over the config section
// which provides access to WriteCache configurations.
type Config config.Config
const (
// SmallSizeDefault is a default size of small objects.
SmallSizeDefault = 32 << 10
// MaxSizeDefault is a default value of the object payload size limit.
MaxSizeDefault = 64 << 20
// WorkersNumberDefault is a default number of workers.
WorkersNumberDefault = 20
// SizeLimitDefault is a default write-cache size limit.
SizeLimitDefault = 1 << 30
)
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Enabled returns true if write-cache is enabled and false otherwise.
//
// Panics if the value is not a boolean.
func (x *Config) Enabled() bool {
return config.Bool((*config.Config)(x), "enabled")
}
// Path returns the value of "path" config parameter.
//
// Panics if the value is not a non-empty string.
func (x *Config) Path() string {
p := config.String(
(*config.Config)(x),
"path",
)
if p == "" {
panic("write cache path not set")
}
return p
}
// SmallObjectSize returns the value of "small_object_size" config parameter.
//
// Returns SmallSizeDefault if the value is not a positive number.
func (x *Config) SmallObjectSize() uint64 {
s := config.SizeInBytesSafe(
(*config.Config)(x),
"small_object_size",
)
if s > 0 {
return s
}
return SmallSizeDefault
}
// MaxObjectSize returns the value of "max_object_size" config parameter.
//
// Returns MaxSizeDefault if the value is not a positive number.
func (x *Config) MaxObjectSize() uint64 {
s := config.SizeInBytesSafe(
(*config.Config)(x),
"max_object_size",
)
if s > 0 {
return s
}
return MaxSizeDefault
}
// WorkersNumber returns the value of "workers_number" config parameter.
//
// Returns WorkersNumberDefault if the value is not a positive number.
func (x *Config) WorkersNumber() int {
c := config.IntSafe(
(*config.Config)(x),
"workers_number",
)
if c > 0 {
return int(c)
}
return WorkersNumberDefault
}
// SizeLimit returns the value of "capacity" config parameter.
//
// Returns SizeLimitDefault if the value is not a positive number.
func (x *Config) SizeLimit() uint64 {
c := config.SizeInBytesSafe(
(*config.Config)(x),
"capacity",
)
if c > 0 {
return c
}
return SizeLimitDefault
}
// NoSync returns the value of "no_sync" config parameter.
//
// Returns false if the value is not a boolean.
func (x *Config) NoSync() bool {
return config.BoolSafe((*config.Config)(x), "no_sync")
}
// BoltDB returns config instance for querying bolt db specific parameters.
func (x *Config) BoltDB() *boltdbconfig.Config {
return (*boltdbconfig.Config)(x)
}

View file

@ -0,0 +1,111 @@
package grpcconfig
import (
"errors"
"strconv"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
var (
errEndpointNotSet = errors.New("empty/not set endpoint, see `grpc.endpoint` section")
errTLSKeyNotSet = errors.New("empty/not set TLS key file path, see `grpc.tls.key` section")
errTLSCertNotSet = errors.New("empty/not set TLS certificate file path, see `grpc.tls.certificate` section")
)
// Config is a wrapper over the config section
// which provides access to gRPC server configurations.
type Config config.Config
// Endpoint returns the value of "endpoint" config parameter.
//
// Panics if the value is not a non-empty string.
func (x *Config) Endpoint() string {
v := config.StringSafe(
(*config.Config)(x),
"endpoint")
if v == "" {
panic(errEndpointNotSet)
}
return v
}
// TLS returns "tls" subsection as a TLSConfig.
//
// Returns nil if "enabled" value of "tls" subsection is false.
func (x *Config) TLS() *TLSConfig {
sub := (*config.Config)(x).
Sub("tls")
if !config.BoolSafe(sub, "enabled") {
return nil
}
return &TLSConfig{
cfg: sub,
}
}
// TLSConfig is a wrapper over the config section
// which provides access to TLS configurations
// of the gRPC server.
type TLSConfig struct {
cfg *config.Config
}
// KeyFile returns the value of "key" config parameter.
//
// Panics if the value is not a non-empty string.
func (tls TLSConfig) KeyFile() string {
v := config.StringSafe(tls.cfg, "key")
if v == "" {
panic(errTLSKeyNotSet)
}
return v
}
// CertificateFile returns the value of "certificate" config parameter.
//
// Panics if the value is not a non-empty string.
func (tls TLSConfig) CertificateFile() string {
v := config.StringSafe(tls.cfg, "certificate")
if v == "" {
panic(errTLSCertNotSet)
}
return v
}
// UseInsecureCrypto returns true if TLS 1.2 cipher suite should not be restricted.
func (tls TLSConfig) UseInsecureCrypto() bool {
return config.BoolSafe(tls.cfg, "use_insecure_crypto")
}
// IterateEndpoints iterates over subsections of "grpc" section of c,
// wrap them into Config and passes to f.
//
// Section names are expected to be consecutive integer numbers, starting from 0.
//
// Panics if N is not a positive number.
func IterateEndpoints(c *config.Config, f func(*Config)) {
c = c.Sub("grpc")
i := uint64(0)
for ; ; i++ {
si := strconv.FormatUint(i, 10)
sc := (*Config)(c.Sub(si))
e := config.StringSafe((*config.Config)(sc), "endpoint")
if e == "" {
break
}
f(sc)
}
if i == 0 {
panic("no gRPC server configured")
}
}

View file

@ -0,0 +1,54 @@
package grpcconfig
import (
"testing"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/stretchr/testify/require"
)
func TestGRPCSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
require.Panics(t, func() {
IterateEndpoints(configtest.EmptyConfig(), nil)
})
})
const path = "../../../../config/example/node"
var fileConfigTest = func(c *config.Config) {
num := 0
IterateEndpoints(c, func(sc *Config) {
defer func() {
num++
}()
tls := sc.TLS()
switch num {
case 0:
require.Equal(t, "s01.frostfs.devenv:8080", sc.Endpoint())
require.NotNil(t, tls)
require.Equal(t, "/path/to/cert", tls.CertificateFile())
require.Equal(t, "/path/to/key", tls.KeyFile())
require.False(t, tls.UseInsecureCrypto())
case 1:
require.Equal(t, "s02.frostfs.devenv:8080", sc.Endpoint())
require.Nil(t, tls)
case 2:
require.Equal(t, "s03.frostfs.devenv:8080", sc.Endpoint())
require.NotNil(t, tls)
require.True(t, tls.UseInsecureCrypto())
}
})
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,22 @@
package internal
import (
"strings"
)
// EnvPrefix is a prefix of ENV variables related
// to storage node configuration.
const EnvPrefix = "neofs"
// EnvSeparator is a section separator in ENV variables.
const EnvSeparator = "_"
// Env returns ENV variable key for a particular config parameter.
func Env(path ...string) string {
return strings.ToUpper(
strings.Join(
append([]string{EnvPrefix}, path...),
EnvSeparator,
),
)
}

View file

@ -0,0 +1,15 @@
package internal_test
import (
"testing"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/internal"
"github.com/stretchr/testify/require"
)
func TestEnv(t *testing.T) {
require.Equal(t,
"NEOFS_SECTION_PARAMETER",
internal.Env("section", "parameter"),
)
}

View file

@ -0,0 +1,26 @@
package loggerconfig
import (
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
const (
// LevelDefault is a default logger level.
LevelDefault = "info"
)
// Level returns the value of "level" config parameter
// from "logger" section.
//
// Returns LevelDefault if the value is not a non-empty string.
func Level(c *config.Config) string {
v := config.StringSafe(
c.Sub("logger"),
"level",
)
if v != "" {
return v
}
return LevelDefault
}

View file

@ -0,0 +1,30 @@
package loggerconfig_test
import (
"testing"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
loggerconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/logger"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/stretchr/testify/require"
)
func TestLoggerSection_Level(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
v := loggerconfig.Level(configtest.EmptyConfig())
require.Equal(t, loggerconfig.LevelDefault, v)
})
const path = "../../../../config/example/node"
var fileConfigTest = func(c *config.Config) {
v := loggerconfig.Level(c)
require.Equal(t, "debug", v)
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,51 @@
package metricsconfig
import (
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
const (
subsection = "prometheus"
// ShutdownTimeoutDefault is a default value for metrics HTTP service timeout.
ShutdownTimeoutDefault = 30 * time.Second
// AddressDefault is a default value for metrics HTTP service endpoint.
AddressDefault = "localhost:9090"
)
// Enabled returns the value of "enabled" config parameter
// from "prometheus" section.
//
// Returns false if the value is missing or invalid.
func Enabled(c *config.Config) bool {
return config.BoolSafe(c.Sub(subsection), "enabled")
}
// ShutdownTimeout returns the value of "shutdown_timeout" config parameter
// from "prometheus" section.
//
// Returns ShutdownTimeoutDefault if the value is not positive duration.
func ShutdownTimeout(c *config.Config) time.Duration {
v := config.DurationSafe(c.Sub(subsection), "shutdown_timeout")
if v > 0 {
return v
}
return ShutdownTimeoutDefault
}
// Address returns the value of "address" config parameter
// from "prometheus" section.
//
// Returns AddressDefault if the value is not set.
func Address(c *config.Config) string {
v := config.StringSafe(c.Sub(subsection), "address")
if v != "" {
return v
}
return AddressDefault
}

View file

@ -0,0 +1,39 @@
package metricsconfig_test
import (
"testing"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
metricsconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/metrics"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/stretchr/testify/require"
)
func TestMetricsSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
to := metricsconfig.ShutdownTimeout(configtest.EmptyConfig())
addr := metricsconfig.Address(configtest.EmptyConfig())
require.Equal(t, metricsconfig.ShutdownTimeoutDefault, to)
require.Equal(t, metricsconfig.AddressDefault, addr)
require.False(t, metricsconfig.Enabled(configtest.EmptyConfig()))
})
const path = "../../../../config/example/node"
var fileConfigTest = func(c *config.Config) {
to := metricsconfig.ShutdownTimeout(c)
addr := metricsconfig.Address(c)
require.Equal(t, 15*time.Second, to)
require.Equal(t, "localhost:9090", addr)
require.True(t, metricsconfig.Enabled(c))
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,99 @@
package morphconfig
import (
"fmt"
"strconv"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/client"
)
const (
subsection = "morph"
notarySubsection = "notary"
// DialTimeoutDefault is a default dial timeout of morph chain client connection.
DialTimeoutDefault = 5 * time.Second
// PriorityDefault is a default endpoint priority for the morph client.
PriorityDefault = 1
// CacheTTLDefault is a default value for cached values TTL.
// It is 0, because actual default depends on block time.
CacheTTLDefault = time.Duration(0)
// SwitchIntervalDefault is a default Neo RPCs switch interval.
SwitchIntervalDefault = 2 * time.Minute
)
// RPCEndpoint returns list of the values of "rpc_endpoint" config parameter
// from "morph" section.
//
// Throws panic if list is empty.
func RPCEndpoint(c *config.Config) []client.Endpoint {
var es []client.Endpoint
sub := c.Sub(subsection).Sub("rpc_endpoint")
for i := 0; ; i++ {
s := sub.Sub(strconv.FormatInt(int64(i), 10))
addr := config.StringSafe(s, "address")
if addr == "" {
break
}
priority := int(config.IntSafe(s, "priority"))
if priority <= 0 {
priority = PriorityDefault
}
es = append(es, client.Endpoint{
Address: addr,
Priority: priority,
})
}
if len(es) == 0 {
panic(fmt.Errorf("no morph chain RPC endpoints, see `morph.rpc_endpoint` section"))
}
return es
}
// DialTimeout returns the value of "dial_timeout" config parameter
// from "morph" section.
//
// Returns DialTimeoutDefault if the value is not positive duration.
func DialTimeout(c *config.Config) time.Duration {
v := config.DurationSafe(c.Sub(subsection), "dial_timeout")
if v > 0 {
return v
}
return DialTimeoutDefault
}
// CacheTTL returns the value of "cache_ttl" config parameter
// from "morph" section.
//
// Returns CacheTTLDefault if value is zero or invalid. Supports negative durations.
func CacheTTL(c *config.Config) time.Duration {
res := config.DurationSafe(c.Sub(subsection), "cache_ttl")
if res != 0 {
return res
}
return CacheTTLDefault
}
// SwitchInterval returns the value of "switch_interval" config parameter
// from "morph" section.
//
// Returns SwitchIntervalDefault if value is not positive duration.
func SwitchInterval(c *config.Config) time.Duration {
res := config.DurationSafe(c.Sub(subsection), "switch_interval")
if res != 0 {
return res
}
return SwitchIntervalDefault
}

View file

@ -0,0 +1,45 @@
package morphconfig_test
import (
"testing"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
morphconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/morph"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/client"
"github.com/stretchr/testify/require"
)
func TestMorphSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
empty := configtest.EmptyConfig()
require.Panics(t, func() { morphconfig.RPCEndpoint(empty) })
require.Equal(t, morphconfig.DialTimeoutDefault, morphconfig.DialTimeout(empty))
require.Equal(t, morphconfig.CacheTTLDefault, morphconfig.CacheTTL(empty))
require.Equal(t, morphconfig.SwitchIntervalDefault, morphconfig.SwitchInterval(empty))
})
const path = "../../../../config/example/node"
var (
rpcs = []client.Endpoint{
{"wss://rpc1.morph.fs.neo.org:40341/ws", 1},
{"wss://rpc2.morph.fs.neo.org:40341/ws", 2},
}
)
var fileConfigTest = func(c *config.Config) {
require.Equal(t, rpcs, morphconfig.RPCEndpoint(c))
require.Equal(t, 30*time.Second, morphconfig.DialTimeout(c))
require.Equal(t, 15*time.Second, morphconfig.CacheTTL(c))
require.Equal(t, 3*time.Minute, morphconfig.SwitchInterval(c))
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,274 @@
package nodeconfig
import (
"fmt"
"os"
"strconv"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
"github.com/TrueCloudLab/frostfs-node/pkg/network"
utilConfig "github.com/TrueCloudLab/frostfs-node/pkg/util/config"
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
)
// PersistentSessionsConfig is a wrapper over "persistent_sessions" config section
// which provides access to persistent session tokens storage configuration of node.
type PersistentSessionsConfig struct {
cfg *config.Config
}
// PersistentStateConfig is a wrapper over "persistent_state" config section
// which provides access to persistent state storage configuration of node.
type PersistentStateConfig struct {
cfg *config.Config
}
// NotificationConfig is a wrapper over "notification" config section
// which provides access to object notification configuration of node.
type NotificationConfig struct {
cfg *config.Config
}
const (
subsection = "node"
persistentSessionsSubsection = "persistent_sessions"
persistentStateSubsection = "persistent_state"
notificationSubsection = "notification"
attributePrefix = "attribute"
// PersistentStatePathDefault is a default path for persistent state file.
PersistentStatePathDefault = ".frostfs-storage-state"
// NotificationTimeoutDefault is a default timeout for object notification operation.
NotificationTimeoutDefault = 5 * time.Second
)
// Key returns the value of "key" config parameter
// from "node" section.
//
// If the value is not set, fallbacks to Wallet section.
//
// Panics if the value is incorrect filename of binary encoded private key.
func Key(c *config.Config) *keys.PrivateKey {
v := config.StringSafe(c.Sub(subsection), "key")
if v == "" {
return Wallet(c)
}
var (
key *keys.PrivateKey
err error
data []byte
)
if data, err = os.ReadFile(v); err == nil {
key, err = keys.NewPrivateKeyFromBytes(data)
}
if err != nil {
panic(fmt.Errorf("invalid private key in node section: %w", err))
}
return key
}
// Wallet returns the value of a node private key from "node" section.
//
// Panics if section contains invalid values.
func Wallet(c *config.Config) *keys.PrivateKey {
v := c.Sub(subsection).Sub("wallet")
acc, err := utilConfig.LoadAccount(
config.String(v, "path"),
config.String(v, "address"),
config.String(v, "password"))
if err != nil {
panic(fmt.Errorf("invalid wallet config: %w", err))
}
return acc.PrivateKey()
}
type stringAddressGroup []string
func (x stringAddressGroup) IterateAddresses(f func(string) bool) {
for i := range x {
if f(x[i]) {
break
}
}
}
func (x stringAddressGroup) NumberOfAddresses() int {
return len(x)
}
// BootstrapAddresses returns the value of "addresses" config parameter
// from "node" section as network.AddressGroup.
//
// Panics if the value is not a string list of valid NeoFS network addresses.
func BootstrapAddresses(c *config.Config) (addr network.AddressGroup) {
v := config.StringSlice(c.Sub(subsection), "addresses")
err := addr.FromIterator(stringAddressGroup(v))
if err != nil {
panic(fmt.Errorf("could not parse bootstrap addresses: %w", err))
}
return addr
}
// Attributes returns list of config parameters
// from "node" section that are set in "attribute_i" format,
// where i in range [0,100).
func Attributes(c *config.Config) (attrs []string) {
const maxAttributes = 100
for i := 0; i < maxAttributes; i++ {
attr := config.StringSafe(c.Sub(subsection), attributePrefix+"_"+strconv.Itoa(i))
if attr == "" {
return
}
attrs = append(attrs, attr)
}
return
}
// Relay returns the value of "relay" config parameter
// from "node" section.
//
// Returns false if the value is not set.
func Relay(c *config.Config) bool {
return config.BoolSafe(c.Sub(subsection), "relay")
}
// PersistentSessions returns structure that provides access to "persistent_sessions"
// subsection of "node" section.
func PersistentSessions(c *config.Config) PersistentSessionsConfig {
return PersistentSessionsConfig{
c.Sub(subsection).Sub(persistentSessionsSubsection),
}
}
// Path returns the value of "path" config parameter.
func (p PersistentSessionsConfig) Path() string {
return config.String(p.cfg, "path")
}
// PersistentState returns structure that provides access to "persistent_state"
// subsection of "node" section.
func PersistentState(c *config.Config) PersistentStateConfig {
return PersistentStateConfig{
c.Sub(subsection).Sub(persistentStateSubsection),
}
}
// Path returns the value of "path" config parameter.
//
// Returns PersistentStatePathDefault if the value is not a non-empty string.
func (p PersistentStateConfig) Path() string {
v := config.String(p.cfg, "path")
if v != "" {
return v
}
return PersistentStatePathDefault
}
// SubnetConfig represents node configuration related to subnets.
type SubnetConfig config.Config
// Init initializes SubnetConfig from "subnet" sub-section of "node" section
// of the root config.
func (x *SubnetConfig) Init(root config.Config) {
*x = SubnetConfig(*root.Sub(subsection).Sub("subnet"))
}
// ExitZero returns the value of "exit_zero" config parameter as bool.
// Returns false if the value can not be cast.
func (x SubnetConfig) ExitZero() bool {
return config.BoolSafe((*config.Config)(&x), "exit_zero")
}
// IterateSubnets casts the value of "entries" config parameter to string slice,
// iterates over all of its elements and passes them to f.
//
// Does nothing if the value can not be cast to string slice.
func (x SubnetConfig) IterateSubnets(f func(string)) {
ids := config.StringSliceSafe((*config.Config)(&x), "entries")
for i := range ids {
f(ids[i])
}
}
// Notification returns structure that provides access to "notification"
// subsection of "node" section.
func Notification(c *config.Config) NotificationConfig {
return NotificationConfig{
c.Sub(subsection).Sub(notificationSubsection),
}
}
// Enabled returns the value of "enabled" config parameter from "notification"
// subsection of "node" section.
//
// Returns false if the value is not presented.
func (n NotificationConfig) Enabled() bool {
return config.BoolSafe(n.cfg, "enabled")
}
// DefaultTopic returns the value of "default_topic" config parameter from
// "notification" subsection of "node" section.
//
// Returns empty string if the value is not presented.
func (n NotificationConfig) DefaultTopic() string {
return config.StringSafe(n.cfg, "default_topic")
}
// Endpoint returns the value of "endpoint" config parameter from "notification"
// subsection of "node" section.
//
// Returns empty string if the value is not presented.
func (n NotificationConfig) Endpoint() string {
return config.StringSafe(n.cfg, "endpoint")
}
// Timeout returns the value of "timeout" config parameter from "notification"
// subsection of "node" section.
//
// Returns NotificationTimeoutDefault if the value is not positive.
func (n NotificationConfig) Timeout() time.Duration {
v := config.DurationSafe(n.cfg, "timeout")
if v > 0 {
return v
}
return NotificationTimeoutDefault
}
// CertPath returns the value of "certificate_path" config parameter from "notification"
// subsection of "node" section.
//
// Returns empty string if the value is not presented.
func (n NotificationConfig) CertPath() string {
return config.StringSafe(n.cfg, "certificate")
}
// KeyPath returns the value of "key_path" config parameter from
// "notification" subsection of "node" section.
//
// Returns empty string if the value is not presented.
func (n NotificationConfig) KeyPath() string {
return config.StringSafe(n.cfg, "key")
}
// CAPath returns the value of "ca_path" config parameter from
// "notification" subsection of "node" section.
//
// Returns empty string if the value is not presented.
func (n NotificationConfig) CAPath() string {
return config.StringSafe(n.cfg, "ca")
}

View file

@ -0,0 +1,167 @@
package nodeconfig
import (
"testing"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/TrueCloudLab/frostfs-node/pkg/network"
"github.com/nspcc-dev/neo-go/pkg/encoding/address"
"github.com/stretchr/testify/require"
)
func TestNodeSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
empty := configtest.EmptyConfig()
require.Panics(
t,
func() {
Key(empty)
},
)
require.Panics(
t,
func() {
BootstrapAddresses(empty)
},
)
attribute := Attributes(empty)
relay := Relay(empty)
persisessionsPath := PersistentSessions(empty).Path()
persistatePath := PersistentState(empty).Path()
notificationDefaultEnabled := Notification(empty).Enabled()
notificationDefaultEndpoint := Notification(empty).Endpoint()
notificationDefaultTimeout := Notification(empty).Timeout()
notificationDefaultTopic := Notification(empty).DefaultTopic()
notificationDefaultCertPath := Notification(empty).CertPath()
notificationDefaultKeyPath := Notification(empty).KeyPath()
notificationDefaultCAPath := Notification(empty).CAPath()
require.Empty(t, attribute)
require.Equal(t, false, relay)
require.Equal(t, "", persisessionsPath)
require.Equal(t, PersistentStatePathDefault, persistatePath)
require.Equal(t, false, notificationDefaultEnabled)
require.Equal(t, "", notificationDefaultEndpoint)
require.Equal(t, NotificationTimeoutDefault, notificationDefaultTimeout)
require.Equal(t, "", notificationDefaultTopic)
require.Equal(t, "", notificationDefaultCertPath)
require.Equal(t, "", notificationDefaultKeyPath)
require.Equal(t, "", notificationDefaultCAPath)
var subnetCfg SubnetConfig
subnetCfg.Init(*empty)
require.False(t, subnetCfg.ExitZero())
called := false
subnetCfg.IterateSubnets(func(string) {
called = true
})
require.False(t, called)
})
const path = "../../../../config/example/node"
var fileConfigTest = func(c *config.Config) {
key := Key(c)
addrs := BootstrapAddresses(c)
attributes := Attributes(c)
relay := Relay(c)
wKey := Wallet(c)
persisessionsPath := PersistentSessions(c).Path()
persistatePath := PersistentState(c).Path()
notificationEnabled := Notification(c).Enabled()
notificationEndpoint := Notification(c).Endpoint()
notificationTimeout := Notification(c).Timeout()
notificationDefaultTopic := Notification(c).DefaultTopic()
notificationCertPath := Notification(c).CertPath()
notificationKeyPath := Notification(c).KeyPath()
notificationCAPath := Notification(c).CAPath()
expectedAddr := []struct {
str string
host string
}{
{
str: "/dns4/localhost/tcp/8083/tls",
host: "grpcs://localhost:8083",
},
{
str: "/dns4/s01.frostfs.devenv/tcp/8080",
host: "s01.frostfs.devenv:8080",
},
{
str: "/dns4/s02.frostfs.devenv/tcp/8081",
host: "s02.frostfs.devenv:8081",
},
{
str: "/ip4/127.0.0.1/tcp/8082",
host: "127.0.0.1:8082",
},
}
require.Equal(t, "NbUgTSFvPmsRxmGeWpuuGeJUoRoi6PErcM", key.Address())
require.EqualValues(t, len(expectedAddr), addrs.Len())
ind := 0
addrs.IterateAddresses(func(addr network.Address) bool {
require.Equal(t, expectedAddr[ind].str, addr.String())
require.Equal(t, expectedAddr[ind].host, addr.URIAddr())
ind++
return false
})
require.Equal(t, true, relay)
require.Len(t, attributes, 2)
require.Equal(t, "Price:11", attributes[0])
require.Equal(t, "UN-LOCODE:RU MSK", attributes[1])
require.NotNil(t, wKey)
require.Equal(t,
config.StringSafe(c.Sub("node").Sub("wallet"), "address"),
address.Uint160ToString(wKey.GetScriptHash()))
require.Equal(t, "/sessions", persisessionsPath)
require.Equal(t, "/state", persistatePath)
require.Equal(t, true, notificationEnabled)
require.Equal(t, "tls://localhost:4222", notificationEndpoint)
require.Equal(t, 6*time.Second, notificationTimeout)
require.Equal(t, "topic", notificationDefaultTopic)
require.Equal(t, "/cert/path", notificationCertPath)
require.Equal(t, "/key/path", notificationKeyPath)
require.Equal(t, "/ca/path", notificationCAPath)
var subnetCfg SubnetConfig
subnetCfg.Init(*c)
require.True(t, subnetCfg.ExitZero())
var ids []string
subnetCfg.IterateSubnets(func(id string) {
ids = append(ids, id)
})
require.Equal(t, []string{"123", "456", "789"}, ids)
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,30 @@
{
"version": "3.0",
"accounts": [
{
"address": "NcpJzXcSDrh5CCizf4K9Ro6w4t59J5LKzz",
"key": "6PYXFRFUfoMNjWd2UmaaEjwHSWpifcLLTbEfhkwXdiSZ2n2WLfr75JpxmJ",
"label": "testacc",
"contract": {
"script": "DCECaeaVhKFa+ENNUpRJLz6BRmRbkIaoN+xZt3VHzlzkHJZBVuezJw==",
"parameters": [
{
"name": "parameter0",
"type": "Signature"
}
],
"deployed": false
},
"lock": false,
"isDefault": false
}
],
"scrypt": {
"n": 16384,
"r": 8,
"p": 8
},
"extra": {
"Tokens": null
}
}

View file

@ -0,0 +1 @@
с╨─ЧдФ╕Я?вьэЁ╡²Оv─/l⌠║еы

View file

@ -0,0 +1,41 @@
package objectconfig
import (
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
// PutConfig is a wrapper over "put" config section which provides access
// to object put pipeline configuration of object service.
type PutConfig struct {
cfg *config.Config
}
const (
subsection = "object"
putSubsection = "put"
// PutPoolSizeDefault is a default value of routine pool size to
// process object.Put requests in object service.
PutPoolSizeDefault = 10
)
// Put returns structure that provides access to "put" subsection of
// "object" section.
func Put(c *config.Config) PutConfig {
return PutConfig{
c.Sub(subsection).Sub(putSubsection),
}
}
// PoolSizeRemote returns the value of "pool_size_remote" config parameter.
//
// Returns PutPoolSizeDefault if the value is not a positive number.
func (g PutConfig) PoolSizeRemote() int {
v := config.Int(g.cfg, "pool_size_remote")
if v > 0 {
return int(v)
}
return PutPoolSizeDefault
}

View file

@ -0,0 +1,30 @@
package objectconfig_test
import (
"testing"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
objectconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/object"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/stretchr/testify/require"
)
func TestObjectSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
empty := configtest.EmptyConfig()
require.Equal(t, objectconfig.PutPoolSizeDefault, objectconfig.Put(empty).PoolSizeRemote())
})
const path = "../../../../config/example/node"
var fileConfigTest = func(c *config.Config) {
require.Equal(t, 100, objectconfig.Put(c).PoolSizeRemote())
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,20 @@
package config
type opts struct {
path string
}
func defaultOpts() *opts {
return new(opts)
}
// Option allows to set an optional parameter of the Config.
type Option func(*opts)
// WithConfigFile returns an option to set the system path
// to the configuration file.
func WithConfigFile(path string) Option {
return func(o *opts) {
o.path = path
}
}

View file

@ -0,0 +1,27 @@
package policerconfig
import (
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
const (
subsection = "policer"
// HeadTimeoutDefault is a default object.Head request timeout in policer.
HeadTimeoutDefault = 5 * time.Second
)
// HeadTimeout returns the value of "head_timeout" config parameter
// from "policer" section.
//
// Returns HeadTimeoutDefault if the value is not positive duration.
func HeadTimeout(c *config.Config) time.Duration {
v := config.DurationSafe(c.Sub(subsection), "head_timeout")
if v > 0 {
return v
}
return HeadTimeoutDefault
}

View file

@ -0,0 +1,31 @@
package policerconfig_test
import (
"testing"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
policerconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/policer"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/stretchr/testify/require"
)
func TestPolicerSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
empty := configtest.EmptyConfig()
require.Equal(t, policerconfig.HeadTimeoutDefault, policerconfig.HeadTimeout(empty))
})
const path = "../../../../config/example/node"
var fileConfigTest = func(c *config.Config) {
require.Equal(t, 15*time.Second, policerconfig.HeadTimeout(c))
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,53 @@
package profilerconfig
import (
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
const (
subsection = "pprof"
// ShutdownTimeoutDefault is a default value for profiler HTTP service timeout.
ShutdownTimeoutDefault = 30 * time.Second
// AddressDefault is a default value for profiler HTTP service endpoint.
AddressDefault = "localhost:6060"
)
// Enabled returns the value of "enabled" config parameter
// from "pprof" section.
//
// Returns false if the value is missing or invalid.
func Enabled(c *config.Config) bool {
return config.BoolSafe(c.Sub(subsection), "enabled")
}
// ShutdownTimeout returns the value of "shutdown_timeout" config parameter
// from "pprof" section.
//
// Returns ShutdownTimeoutDefault if the value is not positive duration.
func ShutdownTimeout(c *config.Config) time.Duration {
v := config.DurationSafe(c.Sub(subsection), "shutdown_timeout")
if v > 0 {
return v
}
return ShutdownTimeoutDefault
}
// Address returns the value of "address" config parameter
// from "pprof" section.
//
// Returns AddressDefault if the value is not set.
func Address(c *config.Config) string {
s := c.Sub(subsection)
v := config.StringSafe(s, "address")
if v != "" {
return v
}
return AddressDefault
}

View file

@ -0,0 +1,39 @@
package profilerconfig_test
import (
"testing"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
profilerconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/profiler"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/stretchr/testify/require"
)
func TestProfilerSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
to := profilerconfig.ShutdownTimeout(configtest.EmptyConfig())
addr := profilerconfig.Address(configtest.EmptyConfig())
require.Equal(t, profilerconfig.ShutdownTimeoutDefault, to)
require.Equal(t, profilerconfig.AddressDefault, addr)
require.False(t, profilerconfig.Enabled(configtest.EmptyConfig()))
})
const path = "../../../../config/example/node"
var fileConfigTest = func(c *config.Config) {
to := profilerconfig.ShutdownTimeout(c)
addr := profilerconfig.Address(c)
require.Equal(t, 15*time.Second, to)
require.Equal(t, "localhost:6060", addr)
require.True(t, profilerconfig.Enabled(c))
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,33 @@
package replicatorconfig
import (
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
const (
subsection = "replicator"
// PutTimeoutDefault is a default timeout of object put request in replicator.
PutTimeoutDefault = 5 * time.Second
)
// PutTimeout returns the value of "put_timeout" config parameter
// from "replicator" section.
//
// Returns PutTimeoutDefault if the value is not positive duration.
func PutTimeout(c *config.Config) time.Duration {
v := config.DurationSafe(c.Sub(subsection), "put_timeout")
if v > 0 {
return v
}
return PutTimeoutDefault
}
// PoolSize returns the value of "pool_size" config parameter
// from "replicator" section.
func PoolSize(c *config.Config) int {
return int(config.IntSafe(c.Sub(subsection), "pool_size"))
}

View file

@ -0,0 +1,33 @@
package replicatorconfig_test
import (
"testing"
"time"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
replicatorconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/replicator"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/stretchr/testify/require"
)
func TestReplicatorSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
empty := configtest.EmptyConfig()
require.Equal(t, replicatorconfig.PutTimeoutDefault, replicatorconfig.PutTimeout(empty))
require.Equal(t, 0, replicatorconfig.PoolSize(empty))
})
const path = "../../../../config/example/node"
var fileConfigTest = func(c *config.Config) {
require.Equal(t, 15*time.Second, replicatorconfig.PutTimeout(c))
require.Equal(t, 10, replicatorconfig.PoolSize(c))
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,80 @@
package configtest
import (
"bufio"
"os"
"strings"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
func fromFile(path string) *config.Config {
var p config.Prm
os.Clearenv() // ENVs have priority over config files, so we do this in tests
return config.New(p,
config.WithConfigFile(path),
)
}
func fromEnvFile(path string) *config.Config {
var p config.Prm
loadEnv(path) // github.com/joho/godotenv can do that as well
return config.New(p)
}
func forEachFile(paths []string, f func(*config.Config)) {
for i := range paths {
f(fromFile(paths[i]))
}
}
// ForEachFileType passes configs read from next files:
// - `<pref>.yaml`;
// - `<pref>.json`.
func ForEachFileType(pref string, f func(*config.Config)) {
forEachFile([]string{
pref + ".yaml",
pref + ".json",
}, f)
}
// ForEnvFileType creates config from `<pref>.env` file.
func ForEnvFileType(pref string, f func(*config.Config)) {
f(fromEnvFile(pref + ".env"))
}
// EmptyConfig returns config without any values and sections.
func EmptyConfig() *config.Config {
var p config.Prm
return config.New(p)
}
// loadEnv reads .env file, parses `X=Y` records and sets OS ENVs.
func loadEnv(path string) {
f, err := os.Open(path)
if err != nil {
panic("can't open .env file")
}
defer f.Close()
scanner := bufio.NewScanner(f)
for scanner.Scan() {
pair := strings.SplitN(scanner.Text(), "=", 2)
if len(pair) != 2 {
continue
}
pair[1] = strings.Trim(pair[1], `"`)
err = os.Setenv(pair[0], pair[1])
if err != nil {
panic("can't set environment variable")
}
}
}

View file

@ -0,0 +1,82 @@
{
"value": "some value",
"section": {
"any": "thing",
"sub": {
"sub": {
"sub1": {
"key": "val1"
},
"sub2": {
"key": "val2"
}
}
}
},
"string_slice": {
"empty": [],
"filled": [
"string1",
"string2"
],
"incorrect": null
},
"string": {
"correct": "some string",
"incorrect": []
},
"duration": {
"correct": "15m",
"incorrect": "some string"
},
"number": {
"int_pos": 1,
"int_neg": -1,
"fract_pos": 2.5,
"fract_neg": -2.5,
"incorrect": "some string"
},
"bool": {
"correct": true,
"correct_string": "true",
"incorrect": "not true"
},
"sizes": {
"size_kb": "1 kb",
"size_kb_no_space": "2kb",
"size_mb": "12m",
"size_gb": "4g",
"size_tb": "5 TB",
"size_float": ".5t",
"size_float_big": "14.123 gb",
"size_i_am_not_very_clever": "12.12345678",
"size_bytes": "2048b",
"size_bytes_no_suffix": 123456
},
"with_default": {
"default": {
"sub": {
"missing": 123,
"overridden": "x"
},
"missing": 42,
"overridden": "a",
"overridden_with_default": true
},
"custom": {
"sub": {
"overridden": "y"
},
"overridden": "b",
"overridden_with_default": false
}
}
}

View file

@ -0,0 +1,68 @@
value: some value
section:
any: thing
sub:
sub:
sub1:
key: val1
sub2:
key: val2
string_slice:
empty: []
filled:
- string1
- string2
incorrect:
string:
correct: some string
incorrect: []
duration:
correct: 15m
incorrect: some string
number:
int_pos: 1
int_neg: -1
fract_pos: 2.5
fract_neg: -2.5
incorrect: some string
bool:
correct: true
correct_string: "true"
incorrect: not true
sizes:
size_kb: 1 kb
size_kb_no_space: 2kb
size_mb: 12m
size_gb: 4g
size_tb: 5 TB
size_float: .5t
size_float_big: 14.123 gb
size_i_am_not_very_clever: 12.12345678
size_bytes: 2048b
size_bytes_no_suffix: 123456
with_default:
default:
sub:
missing: 123
overridden: "x"
missing: 42
overridden: "a"
overridden_with_default: true
custom:
sub:
overridden: "y"
overridden: "b"
overridden_with_default: false

View file

@ -0,0 +1,53 @@
package treeconfig
import "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
const (
subsection = "tree"
)
// TreeConfig is a wrapper over "tree" config section
// which provides access to the configuration of the tree service.
type TreeConfig struct {
cfg *config.Config
}
// Tree returns structure that provides access to a "tree"
// configuration subsection.
func Tree(c *config.Config) TreeConfig {
return TreeConfig{
c.Sub(subsection),
}
}
// Enabled returns the value of "enabled" config parameter
// from the "tree" section.
//
// Returns `false` if config value is not specified.
func (c TreeConfig) Enabled() bool {
return config.BoolSafe(c.cfg, "enabled")
}
// CacheSize returns the value of "cache_size" config parameter
// from the "tree" section.
//
// Returns `0` if config value is not specified.
func (c TreeConfig) CacheSize() int {
return int(config.IntSafe(c.cfg, "cache_size"))
}
// ReplicationChannelCapacity returns the value of "replication_channel_capacity"
// config parameter from the "tree" section.
//
// Returns `0` if config value is not specified.
func (c TreeConfig) ReplicationChannelCapacity() int {
return int(config.IntSafe(c.cfg, "replication_channel_capacity"))
}
// ReplicationWorkerCount returns the value of "replication_worker_count"
// config parameter from the "tree" section.
//
// Returns `0` if config value is not specified.
func (c TreeConfig) ReplicationWorkerCount() int {
return int(config.IntSafe(c.cfg, "replication_worker_count"))
}

View file

@ -0,0 +1,40 @@
package treeconfig_test
import (
"testing"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
treeconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/tree"
"github.com/stretchr/testify/require"
)
func TestTreeSection(t *testing.T) {
t.Run("defaults", func(t *testing.T) {
empty := configtest.EmptyConfig()
treeSec := treeconfig.Tree(empty)
require.False(t, treeSec.Enabled())
require.Equal(t, 0, treeSec.CacheSize())
require.Equal(t, 0, treeSec.ReplicationChannelCapacity())
require.Equal(t, 0, treeSec.ReplicationWorkerCount())
})
const path = "../../../../config/example/node"
var fileConfigTest = func(c *config.Config) {
treeSec := treeconfig.Tree(c)
require.True(t, treeSec.Enabled())
require.Equal(t, 15, treeSec.CacheSize())
require.Equal(t, 32, treeSec.ReplicationChannelCapacity())
require.Equal(t, 32, treeSec.ReplicationWorkerCount())
}
configtest.ForEachFileType(path, fileConfigTest)
t.Run("ENV", func(t *testing.T) {
configtest.ForEnvFileType(path, fileConfigTest)
})
}

View file

@ -0,0 +1,684 @@
package main
import (
"bytes"
"context"
"crypto/ecdsa"
"crypto/sha256"
"errors"
"fmt"
"strconv"
containerV2 "github.com/TrueCloudLab/frostfs-api-go/v2/container"
containerGRPC "github.com/TrueCloudLab/frostfs-api-go/v2/container/grpc"
"github.com/TrueCloudLab/frostfs-node/pkg/core/client"
containerCore "github.com/TrueCloudLab/frostfs-node/pkg/core/container"
netmapCore "github.com/TrueCloudLab/frostfs-node/pkg/core/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/engine"
cntClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client/container"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/event"
containerEvent "github.com/TrueCloudLab/frostfs-node/pkg/morph/event/container"
containerTransportGRPC "github.com/TrueCloudLab/frostfs-node/pkg/network/transport/container/grpc"
containerService "github.com/TrueCloudLab/frostfs-node/pkg/services/container"
loadcontroller "github.com/TrueCloudLab/frostfs-node/pkg/services/container/announcement/load/controller"
loadroute "github.com/TrueCloudLab/frostfs-node/pkg/services/container/announcement/load/route"
placementrouter "github.com/TrueCloudLab/frostfs-node/pkg/services/container/announcement/load/route/placement"
loadstorage "github.com/TrueCloudLab/frostfs-node/pkg/services/container/announcement/load/storage"
containerMorph "github.com/TrueCloudLab/frostfs-node/pkg/services/container/morph"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
apiClient "github.com/TrueCloudLab/frostfs-sdk-go/client"
containerSDK "github.com/TrueCloudLab/frostfs-sdk-go/container"
cid "github.com/TrueCloudLab/frostfs-sdk-go/container/id"
"github.com/TrueCloudLab/frostfs-sdk-go/netmap"
"github.com/TrueCloudLab/frostfs-sdk-go/user"
"go.uber.org/zap"
)
const (
startEstimationNotifyEvent = "StartEstimation"
stopEstimationNotifyEvent = "StopEstimation"
)
func initContainerService(c *cfg) {
// container wrapper that tries to invoke notary
// requests if chain is configured so
wrap, err := cntClient.NewFromMorph(c.cfgMorph.client, c.cfgContainer.scriptHash, 0, cntClient.TryNotary())
fatalOnErr(err)
c.shared.cnrClient = wrap
// container wrapper that always sends non-notary
// requests
wrapperNoNotary, err := cntClient.NewFromMorph(c.cfgMorph.client, c.cfgContainer.scriptHash, 0)
fatalOnErr(err)
cnrSrc := cntClient.AsContainerSource(wrap)
eACLFetcher := &morphEACLFetcher{
w: wrap,
}
cnrRdr := new(morphContainerReader)
cnrWrt := &morphContainerWriter{
neoClient: wrap,
}
if c.cfgMorph.cacheTTL <= 0 {
c.cfgObject.eaclSource = eACLFetcher
cnrRdr.eacl = eACLFetcher
c.cfgObject.cnrSource = cnrSrc
cnrRdr.get = cnrSrc
cnrRdr.lister = wrap
} else {
// use RPC node as source of Container contract items (with caching)
cachedContainerStorage := newCachedContainerStorage(cnrSrc, c.cfgMorph.cacheTTL)
cachedEACLStorage := newCachedEACLStorage(eACLFetcher, c.cfgMorph.cacheTTL)
cachedContainerLister := newCachedContainerLister(wrap, c.cfgMorph.cacheTTL)
subscribeToContainerCreation(c, func(e event.Event) {
ev := e.(containerEvent.PutSuccess)
// read owner of the created container in order to update the reading cache.
// TODO: use owner directly from the event after neofs-contract#256 will become resolved
// but don't forget about the profit of reading the new container and caching it:
// creation success are most commonly tracked by polling GET op.
cnr, err := cachedContainerStorage.Get(ev.ID)
if err == nil {
cachedContainerLister.update(cnr.Value.Owner(), ev.ID, true)
} else {
// unlike removal, we expect successful receive of the container
// after successful creation, so logging can be useful
c.log.Error("read newly created container after the notification",
zap.Stringer("id", ev.ID),
zap.Error(err),
)
}
c.log.Debug("container creation event's receipt",
zap.Stringer("id", ev.ID),
)
})
subscribeToContainerRemoval(c, func(e event.Event) {
ev := e.(containerEvent.DeleteSuccess)
// read owner of the removed container in order to update the listing cache.
// It's strange to read already removed container, but we can successfully hit
// the cache.
// TODO: use owner directly from the event after neofs-contract#256 will become resolved
cnr, err := cachedContainerStorage.Get(ev.ID)
if err == nil {
cachedContainerLister.update(cnr.Value.Owner(), ev.ID, false)
}
cachedContainerStorage.handleRemoval(ev.ID)
c.log.Debug("container removal event's receipt",
zap.Stringer("id", ev.ID),
)
})
c.cfgObject.eaclSource = cachedEACLStorage
c.cfgObject.cnrSource = cachedContainerStorage
cnrRdr.lister = cachedContainerLister
cnrRdr.eacl = c.cfgObject.eaclSource
cnrRdr.get = c.cfgObject.cnrSource
cnrWrt.cacheEnabled = true
cnrWrt.lists = cachedContainerLister
cnrWrt.eacls = cachedEACLStorage
}
localMetrics := &localStorageLoad{
log: c.log,
engine: c.cfgObject.cfgLocalStorage.localStorage,
}
pubKey := c.key.PublicKey().Bytes()
resultWriter := &morphLoadWriter{
log: c.log,
cnrMorphClient: wrapperNoNotary,
key: pubKey,
}
loadAccumulator := loadstorage.New(loadstorage.Prm{})
loadPlacementBuilder := &loadPlacementBuilder{
log: c.log,
nmSrc: c.netMapSource,
cnrSrc: cnrSrc,
}
routeBuilder := placementrouter.New(placementrouter.Prm{
PlacementBuilder: loadPlacementBuilder,
})
loadRouter := loadroute.New(
loadroute.Prm{
LocalServerInfo: c,
RemoteWriterProvider: &remoteLoadAnnounceProvider{
key: &c.key.PrivateKey,
netmapKeys: c,
clientCache: c.bgClientCache,
deadEndProvider: loadcontroller.SimpleWriterProvider(loadAccumulator),
},
Builder: routeBuilder,
},
loadroute.WithLogger(c.log),
)
ctrl := loadcontroller.New(
loadcontroller.Prm{
LocalMetrics: loadcontroller.SimpleIteratorProvider(localMetrics),
AnnouncementAccumulator: loadcontroller.SimpleIteratorProvider(loadAccumulator),
LocalAnnouncementTarget: loadRouter,
ResultReceiver: loadcontroller.SimpleWriterProvider(resultWriter),
},
loadcontroller.WithLogger(c.log),
)
setContainerNotificationParser(c, startEstimationNotifyEvent, containerEvent.ParseStartEstimation)
addContainerAsyncNotificationHandler(c, startEstimationNotifyEvent, func(ev event.Event) {
ctrl.Start(loadcontroller.StartPrm{
Epoch: ev.(containerEvent.StartEstimation).Epoch(),
})
})
setContainerNotificationParser(c, stopEstimationNotifyEvent, containerEvent.ParseStopEstimation)
addContainerAsyncNotificationHandler(c, stopEstimationNotifyEvent, func(ev event.Event) {
ctrl.Stop(loadcontroller.StopPrm{
Epoch: ev.(containerEvent.StopEstimation).Epoch(),
})
})
server := containerTransportGRPC.New(
containerService.NewSignService(
&c.key.PrivateKey,
containerService.NewResponseService(
&usedSpaceService{
Server: containerService.NewExecutionService(containerMorph.NewExecutor(cnrRdr, cnrWrt)),
loadWriterProvider: loadRouter,
loadPlacementBuilder: loadPlacementBuilder,
routeBuilder: routeBuilder,
cfg: c,
},
c.respSvc,
),
),
)
for _, srv := range c.cfgGRPC.servers {
containerGRPC.RegisterContainerServiceServer(srv, server)
}
}
// addContainerNotificationHandler adds handler that will be executed synchronously.
func addContainerNotificationHandler(c *cfg, sTyp string, h event.Handler) {
typ := event.TypeFromString(sTyp)
if c.cfgContainer.subscribers == nil {
c.cfgContainer.subscribers = make(map[event.Type][]event.Handler, 1)
}
c.cfgContainer.subscribers[typ] = append(c.cfgContainer.subscribers[typ], h)
}
// addContainerAsyncNotificationHandler adds handler that will be executed asynchronously via container workerPool.
func addContainerAsyncNotificationHandler(c *cfg, sTyp string, h event.Handler) {
addContainerNotificationHandler(
c,
sTyp,
event.WorkerPoolHandler(
c.cfgContainer.workerPool,
h,
c.log,
),
)
}
// stores already registered parsers of the notification events thrown by Container contract.
// MUST NOT be used concurrently.
var mRegisteredParsersContainer = make(map[string]struct{})
// registers event parser by name once. MUST NOT be called concurrently.
func registerEventParserOnceContainer(c *cfg, name string, p event.NotificationParser) {
if _, ok := mRegisteredParsersContainer[name]; !ok {
setContainerNotificationParser(c, name, p)
mRegisteredParsersContainer[name] = struct{}{}
}
}
// subscribes to successful container creation. Provided handler is called asynchronously
// on corresponding routine pool. MUST NOT be called concurrently with itself and other
// similar functions.
func subscribeToContainerCreation(c *cfg, h event.Handler) {
const eventNameContainerCreated = "PutSuccess"
registerEventParserOnceContainer(c, eventNameContainerCreated, containerEvent.ParsePutSuccess)
addContainerAsyncNotificationHandler(c, eventNameContainerCreated, h)
}
// like subscribeToContainerCreation but for removal.
func subscribeToContainerRemoval(c *cfg, h event.Handler) {
const eventNameContainerRemoved = "DeleteSuccess"
registerEventParserOnceContainer(c, eventNameContainerRemoved, containerEvent.ParseDeleteSuccess)
addContainerAsyncNotificationHandler(c, eventNameContainerRemoved, h)
}
func setContainerNotificationParser(c *cfg, sTyp string, p event.NotificationParser) {
typ := event.TypeFromString(sTyp)
if c.cfgContainer.parsers == nil {
c.cfgContainer.parsers = make(map[event.Type]event.NotificationParser, 1)
}
c.cfgContainer.parsers[typ] = p
}
type morphLoadWriter struct {
log *logger.Logger
cnrMorphClient *cntClient.Client
key []byte
}
func (w *morphLoadWriter) Put(a containerSDK.SizeEstimation) error {
w.log.Debug("save used space announcement in contract",
zap.Uint64("epoch", a.Epoch()),
zap.Stringer("cid", a.Container()),
zap.Uint64("size", a.Value()),
)
prm := cntClient.AnnounceLoadPrm{}
prm.SetAnnouncement(a)
prm.SetReporter(w.key)
return w.cnrMorphClient.AnnounceLoad(prm)
}
func (*morphLoadWriter) Close() error {
return nil
}
type nopLoadWriter struct{}
func (nopLoadWriter) Put(containerSDK.SizeEstimation) error {
return nil
}
func (nopLoadWriter) Close() error {
return nil
}
type remoteLoadAnnounceProvider struct {
key *ecdsa.PrivateKey
netmapKeys netmapCore.AnnouncedKeys
clientCache interface {
Get(client.NodeInfo) (client.Client, error)
}
deadEndProvider loadcontroller.WriterProvider
}
func (r *remoteLoadAnnounceProvider) InitRemote(srv loadroute.ServerInfo) (loadcontroller.WriterProvider, error) {
if srv == nil {
return r.deadEndProvider, nil
}
if r.netmapKeys.IsLocalKey(srv.PublicKey()) {
// if local => return no-op writer
return loadcontroller.SimpleWriterProvider(new(nopLoadWriter)), nil
}
var info client.NodeInfo
err := client.NodeInfoFromRawNetmapElement(&info, srv)
if err != nil {
return nil, fmt.Errorf("parse client node info: %w", err)
}
c, err := r.clientCache.Get(info)
if err != nil {
return nil, fmt.Errorf("could not initialize API client: %w", err)
}
return &remoteLoadAnnounceWriterProvider{
client: c,
}, nil
}
type remoteLoadAnnounceWriterProvider struct {
client client.Client
}
func (p *remoteLoadAnnounceWriterProvider) InitWriter(ctx context.Context) (loadcontroller.Writer, error) {
return &remoteLoadAnnounceWriter{
ctx: ctx,
client: p.client,
}, nil
}
type remoteLoadAnnounceWriter struct {
ctx context.Context
client client.Client
buf []containerSDK.SizeEstimation
}
func (r *remoteLoadAnnounceWriter) Put(a containerSDK.SizeEstimation) error {
r.buf = append(r.buf, a)
return nil
}
func (r *remoteLoadAnnounceWriter) Close() error {
var cliPrm apiClient.PrmAnnounceSpace
cliPrm.SetValues(r.buf)
_, err := r.client.ContainerAnnounceUsedSpace(r.ctx, cliPrm)
return err
}
type loadPlacementBuilder struct {
log *logger.Logger
nmSrc netmapCore.Source
cnrSrc containerCore.Source
}
func (l *loadPlacementBuilder) BuildPlacement(epoch uint64, cnr cid.ID) ([][]netmap.NodeInfo, error) {
cnrNodes, nm, err := l.buildPlacement(epoch, cnr)
if err != nil {
return nil, err
}
const pivotPrefix = "load_announcement_"
pivot := []byte(
pivotPrefix + strconv.FormatUint(epoch, 10),
)
placement, err := nm.PlacementVectors(cnrNodes, pivot)
if err != nil {
return nil, fmt.Errorf("could not build placement vectors: %w", err)
}
return placement, nil
}
func (l *loadPlacementBuilder) buildPlacement(epoch uint64, idCnr cid.ID) ([][]netmap.NodeInfo, *netmap.NetMap, error) {
cnr, err := l.cnrSrc.Get(idCnr)
if err != nil {
return nil, nil, err
}
nm, err := l.nmSrc.GetNetMapByEpoch(epoch)
if err != nil {
return nil, nil, fmt.Errorf("could not get network map: %w", err)
}
binCnr := make([]byte, sha256.Size)
idCnr.Encode(binCnr)
cnrNodes, err := nm.ContainerNodes(cnr.Value.PlacementPolicy(), binCnr)
if err != nil {
return nil, nil, fmt.Errorf("could not build container nodes: %w", err)
}
return cnrNodes, nm, nil
}
type localStorageLoad struct {
log *logger.Logger
engine *engine.StorageEngine
}
func (d *localStorageLoad) Iterate(f loadcontroller.UsedSpaceFilter, h loadcontroller.UsedSpaceHandler) error {
idList, err := engine.ListContainers(d.engine)
if err != nil {
return fmt.Errorf("list containers on engine failure: %w", err)
}
for i := range idList {
sz, err := engine.ContainerSize(d.engine, idList[i])
if err != nil {
d.log.Debug("failed to calculate container size in storage engine",
zap.Stringer("cid", idList[i]),
zap.String("error", err.Error()),
)
continue
}
d.log.Debug("container size in storage engine calculated successfully",
zap.Uint64("size", sz),
zap.Stringer("cid", idList[i]),
)
var a containerSDK.SizeEstimation
a.SetContainer(idList[i])
a.SetValue(sz)
if f != nil && !f(a) {
continue
}
if err := h(a); err != nil {
return err
}
}
return nil
}
type usedSpaceService struct {
containerService.Server
loadWriterProvider loadcontroller.WriterProvider
loadPlacementBuilder *loadPlacementBuilder
routeBuilder loadroute.Builder
cfg *cfg
}
func (c *cfg) PublicKey() []byte {
return nodeKeyFromNetmap(c)
}
func (c *cfg) IsLocalKey(key []byte) bool {
return bytes.Equal(key, c.PublicKey())
}
func (c *cfg) IterateAddresses(f func(string) bool) {
c.iterateNetworkAddresses(f)
}
func (c *cfg) NumberOfAddresses() int {
return c.addressNum()
}
func (c *cfg) ExternalAddresses() []string {
return c.cfgNodeInfo.localInfo.ExternalAddresses()
}
func (c *usedSpaceService) PublicKey() []byte {
return nodeKeyFromNetmap(c.cfg)
}
func (c *usedSpaceService) IterateAddresses(f func(string) bool) {
c.cfg.iterateNetworkAddresses(f)
}
func (c *usedSpaceService) NumberOfAddresses() int {
return c.cfg.addressNum()
}
func (c *usedSpaceService) ExternalAddresses() []string {
return c.cfg.ExternalAddresses()
}
func (c *usedSpaceService) AnnounceUsedSpace(ctx context.Context, req *containerV2.AnnounceUsedSpaceRequest) (*containerV2.AnnounceUsedSpaceResponse, error) {
var passedRoute []loadroute.ServerInfo
for hdr := req.GetVerificationHeader(); hdr != nil; hdr = hdr.GetOrigin() {
passedRoute = append(passedRoute, &containerOnlyKeyRemoteServerInfo{
key: hdr.GetBodySignature().GetKey(),
})
}
for left, right := 0, len(passedRoute)-1; left < right; left, right = left+1, right-1 {
passedRoute[left], passedRoute[right] = passedRoute[right], passedRoute[left]
}
passedRoute = append(passedRoute, c)
w, err := c.loadWriterProvider.InitWriter(loadroute.NewRouteContext(ctx, passedRoute))
if err != nil {
return nil, fmt.Errorf("could not initialize container's used space writer: %w", err)
}
var est containerSDK.SizeEstimation
for _, aV2 := range req.GetBody().GetAnnouncements() {
err = est.ReadFromV2(aV2)
if err != nil {
return nil, fmt.Errorf("invalid size announcement: %w", err)
}
if err := c.processLoadValue(ctx, est, passedRoute, w); err != nil {
return nil, err
}
}
respBody := new(containerV2.AnnounceUsedSpaceResponseBody)
resp := new(containerV2.AnnounceUsedSpaceResponse)
resp.SetBody(respBody)
return resp, nil
}
var errNodeOutsideContainer = errors.New("node outside the container")
type containerOnlyKeyRemoteServerInfo struct {
key []byte
}
func (i *containerOnlyKeyRemoteServerInfo) PublicKey() []byte {
return i.key
}
func (*containerOnlyKeyRemoteServerInfo) IterateAddresses(func(string) bool) {
}
func (*containerOnlyKeyRemoteServerInfo) NumberOfAddresses() int {
return 0
}
func (*containerOnlyKeyRemoteServerInfo) ExternalAddresses() []string {
return nil
}
func (l *loadPlacementBuilder) isNodeFromContainerKey(epoch uint64, cnr cid.ID, key []byte) (bool, error) {
cnrNodes, _, err := l.buildPlacement(epoch, cnr)
if err != nil {
return false, err
}
for i := range cnrNodes {
for j := range cnrNodes[i] {
if bytes.Equal(cnrNodes[i][j].PublicKey(), key) {
return true, nil
}
}
}
return false, nil
}
func (c *usedSpaceService) processLoadValue(_ context.Context, a containerSDK.SizeEstimation,
route []loadroute.ServerInfo, w loadcontroller.Writer) error {
fromCnr, err := c.loadPlacementBuilder.isNodeFromContainerKey(a.Epoch(), a.Container(), route[0].PublicKey())
if err != nil {
return fmt.Errorf("could not verify that the sender belongs to the container: %w", err)
} else if !fromCnr {
return errNodeOutsideContainer
}
err = loadroute.CheckRoute(c.routeBuilder, a, route)
if err != nil {
return fmt.Errorf("wrong route of container's used space value: %w", err)
}
err = w.Put(a)
if err != nil {
return fmt.Errorf("could not write container's used space value: %w", err)
}
return nil
}
// implements interface required by container service provided by morph executor.
type morphContainerReader struct {
eacl containerCore.EACLSource
get containerCore.Source
lister interface {
List(*user.ID) ([]cid.ID, error)
}
}
func (x *morphContainerReader) Get(id cid.ID) (*containerCore.Container, error) {
return x.get.Get(id)
}
func (x *morphContainerReader) GetEACL(id cid.ID) (*containerCore.EACL, error) {
return x.eacl.GetEACL(id)
}
func (x *morphContainerReader) List(id *user.ID) ([]cid.ID, error) {
return x.lister.List(id)
}
type morphContainerWriter struct {
neoClient *cntClient.Client
cacheEnabled bool
eacls *ttlEACLStorage
lists *ttlContainerLister
}
func (m morphContainerWriter) Put(cnr containerCore.Container) (*cid.ID, error) {
return cntClient.Put(m.neoClient, cnr)
}
func (m morphContainerWriter) Delete(witness containerCore.RemovalWitness) error {
return cntClient.Delete(m.neoClient, witness)
}
func (m morphContainerWriter) PutEACL(eaclInfo containerCore.EACL) error {
err := cntClient.PutEACL(m.neoClient, eaclInfo)
if err != nil {
return err
}
if m.cacheEnabled {
id, _ := eaclInfo.Value.CID()
m.eacls.InvalidateEACL(id)
}
return nil
}

View file

@ -0,0 +1,88 @@
package main
import (
"context"
"net"
controlconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/control"
"github.com/TrueCloudLab/frostfs-node/pkg/services/control"
controlSvc "github.com/TrueCloudLab/frostfs-node/pkg/services/control/server"
"github.com/TrueCloudLab/frostfs-node/pkg/services/tree"
cid "github.com/TrueCloudLab/frostfs-sdk-go/container/id"
"go.uber.org/zap"
"google.golang.org/grpc"
)
type treeSynchronizer struct {
treeSvc *tree.Service
}
func (t treeSynchronizer) Synchronize(ctx context.Context, cnr cid.ID, treeID string) error {
return t.treeSvc.SynchronizeTree(ctx, cnr, treeID)
}
func initControlService(c *cfg) {
endpoint := controlconfig.GRPC(c.appCfg).Endpoint()
if endpoint == controlconfig.GRPCEndpointDefault {
return
}
pubs := controlconfig.AuthorizedKeys(c.appCfg)
rawPubs := make([][]byte, 0, len(pubs)+1) // +1 for node key
rawPubs = append(rawPubs, c.key.PublicKey().Bytes())
for i := range pubs {
rawPubs = append(rawPubs, pubs[i].Bytes())
}
ctlSvc := controlSvc.New(
controlSvc.WithKey(&c.key.PrivateKey),
controlSvc.WithAuthorizedKeys(rawPubs),
controlSvc.WithHealthChecker(c),
controlSvc.WithNetMapSource(c.netMapSource),
controlSvc.WithContainerSource(c.cfgObject.cnrSource),
controlSvc.WithReplicator(c.replicator),
controlSvc.WithNodeState(c),
controlSvc.WithLocalStorage(c.cfgObject.cfgLocalStorage.localStorage),
controlSvc.WithTreeService(treeSynchronizer{
c.treeService,
}),
)
lis, err := net.Listen("tcp", endpoint)
if err != nil {
c.log.Error("can't listen gRPC endpoint (control)", zap.Error(err))
return
}
c.cfgControlService.server = grpc.NewServer()
c.onShutdown(func() {
stopGRPC("NeoFS Control API", c.cfgControlService.server, c.log)
})
control.RegisterControlServiceServer(c.cfgControlService.server, ctlSvc)
c.workers = append(c.workers, newWorkerFromFunc(func(ctx context.Context) {
runAndLog(c, "control", false, func(c *cfg) {
fatalOnErr(c.cfgControlService.server.Serve(lis))
})
}))
}
func (c *cfg) NetmapStatus() control.NetmapStatus {
return c.cfgNetmap.state.controlNetmapStatus()
}
func (c *cfg) setHealthStatus(st control.HealthStatus) {
c.healthStatus.Store(int32(st))
if c.metricsCollector != nil {
c.metricsCollector.SetHealth(int32(st))
}
}
func (c *cfg) HealthStatus() control.HealthStatus {
return control.HealthStatus(c.healthStatus.Load())
}

127
cmd/frostfs-node/grpc.go Normal file
View file

@ -0,0 +1,127 @@
package main
import (
"crypto/tls"
"errors"
"fmt"
"net"
"time"
grpcconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/grpc"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
)
func initGRPC(c *cfg) {
var successCount int
grpcconfig.IterateEndpoints(c.appCfg, func(sc *grpcconfig.Config) {
serverOpts := []grpc.ServerOption{
grpc.MaxSendMsgSize(maxMsgSize),
}
tlsCfg := sc.TLS()
if tlsCfg != nil {
cert, err := tls.LoadX509KeyPair(tlsCfg.CertificateFile(), tlsCfg.KeyFile())
if err != nil {
c.log.Error("could not read certificate from file", zap.Error(err))
return
}
var cipherSuites []uint16
if !tlsCfg.UseInsecureCrypto() {
// This more or less follows the list in https://wiki.mozilla.org/Security/Server_Side_TLS
// excluding:
// 1. TLS 1.3 suites need not be specified here.
// 2. Suites that use DH key exchange are not implemented by stdlib.
cipherSuites = []uint16{
tls.TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,
tls.TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,
tls.TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384,
tls.TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,
tls.TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256,
tls.TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256,
}
}
creds := credentials.NewTLS(&tls.Config{
MinVersion: tls.VersionTLS12,
CipherSuites: cipherSuites,
Certificates: []tls.Certificate{cert},
})
serverOpts = append(serverOpts, grpc.Creds(creds))
}
lis, err := net.Listen("tcp", sc.Endpoint())
if err != nil {
c.log.Error("can't listen gRPC endpoint", zap.Error(err))
return
}
c.cfgGRPC.listeners = append(c.cfgGRPC.listeners, lis)
srv := grpc.NewServer(serverOpts...)
c.onShutdown(func() {
stopGRPC("NeoFS Public API", srv, c.log)
})
c.cfgGRPC.servers = append(c.cfgGRPC.servers, srv)
successCount++
})
if successCount == 0 {
fatalOnErr(errors.New("could not listen to any gRPC endpoints"))
}
}
func serveGRPC(c *cfg) {
for i := range c.cfgGRPC.servers {
c.wg.Add(1)
srv := c.cfgGRPC.servers[i]
lis := c.cfgGRPC.listeners[i]
go func() {
defer func() {
c.log.Info("stop listening gRPC endpoint",
zap.String("endpoint", lis.Addr().String()),
)
c.wg.Done()
}()
c.log.Info("start listening gRPC endpoint",
zap.String("endpoint", lis.Addr().String()),
)
if err := srv.Serve(lis); err != nil {
fmt.Println("gRPC server error", err)
}
}()
}
}
func stopGRPC(name string, s *grpc.Server, l *logger.Logger) {
l = &logger.Logger{Logger: l.With(zap.String("name", name))}
l.Info("stopping gRPC server...")
// GracefulStop() may freeze forever, see #1270
done := make(chan struct{})
go func() {
s.GracefulStop()
close(done)
}()
select {
case <-done:
case <-time.After(1 * time.Minute):
l.Info("gRPC cannot shutdown gracefully, forcing stop")
s.Stop()
}
l.Info("gRPC server stopped successfully")
}

154
cmd/frostfs-node/main.go Normal file
View file

@ -0,0 +1,154 @@
package main
import (
"context"
"flag"
"fmt"
"log"
"os"
"os/signal"
"syscall"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
"github.com/TrueCloudLab/frostfs-node/misc"
"github.com/TrueCloudLab/frostfs-node/pkg/services/control"
"go.uber.org/zap"
)
const (
// SuccessReturnCode returns when application closed without panic.
SuccessReturnCode = 0
)
// prints err to standard logger and calls os.Exit(1).
func fatalOnErr(err error) {
if err != nil {
log.Fatal(err)
}
}
// prints err with details to standard logger and calls os.Exit(1).
func fatalOnErrDetails(details string, err error) {
if err != nil {
log.Fatal(fmt.Errorf("%s: %w", details, err))
}
}
func main() {
configFile := flag.String("config", "", "path to config")
versionFlag := flag.Bool("version", false, "frostfs node version")
dryRunFlag := flag.Bool("check", false, "validate configuration and exit")
flag.Parse()
if *versionFlag {
fmt.Print(misc.BuildInfo("NeoFS Storage node"))
os.Exit(SuccessReturnCode)
}
appCfg := config.New(config.Prm{}, config.WithConfigFile(*configFile))
err := validateConfig(appCfg)
fatalOnErr(err)
if *dryRunFlag {
return
}
c := initCfg(appCfg)
initApp(c)
c.setHealthStatus(control.HealthStatus_STARTING)
bootUp(c)
c.setHealthStatus(control.HealthStatus_READY)
wait(c)
c.setHealthStatus(control.HealthStatus_SHUTTING_DOWN)
shutdown(c)
}
func initAndLog(c *cfg, name string, initializer func(*cfg)) {
c.log.Info(fmt.Sprintf("initializing %s service...", name))
initializer(c)
c.log.Info(fmt.Sprintf("%s service has been successfully initialized", name))
}
func initApp(c *cfg) {
initLocalStorage(c)
c.ctx, c.ctxCancel = signal.NotifyContext(context.Background(), syscall.SIGINT, syscall.SIGTERM)
initAndLog(c, "storage engine", func(c *cfg) {
fatalOnErr(c.cfgObject.cfgLocalStorage.localStorage.Open())
fatalOnErr(c.cfgObject.cfgLocalStorage.localStorage.Init())
})
initAndLog(c, "gRPC", initGRPC)
initAndLog(c, "netmap", initNetmapService)
initAndLog(c, "accounting", initAccountingService)
initAndLog(c, "container", initContainerService)
initAndLog(c, "session", initSessionService)
initAndLog(c, "reputation", initReputationService)
initAndLog(c, "notification", initNotifications)
initAndLog(c, "object", initObjectService)
initAndLog(c, "pprof", initProfiler)
initAndLog(c, "prometheus", initMetrics)
initAndLog(c, "tree", initTreeService)
initAndLog(c, "control", initControlService)
initAndLog(c, "morph notifications", listenMorphNotifications)
c.workers = append(c.workers, newWorkerFromFunc(c.configWatcher))
}
func runAndLog(c *cfg, name string, logSuccess bool, starter func(*cfg)) {
c.log.Info(fmt.Sprintf("starting %s service...", name))
starter(c)
if logSuccess {
c.log.Info(fmt.Sprintf("%s service started successfully", name))
}
}
func bootUp(c *cfg) {
runAndLog(c, "NATS", true, connectNats)
runAndLog(c, "gRPC", false, serveGRPC)
runAndLog(c, "notary", true, makeAndWaitNotaryDeposit)
bootstrapNode(c)
startWorkers(c)
}
func wait(c *cfg) {
c.log.Info("application started",
zap.String("version", misc.Version))
select {
case <-c.ctx.Done(): // graceful shutdown
case err := <-c.internalErr: // internal application error
close(c.internalErr)
c.ctxCancel()
c.log.Warn("internal application error",
zap.String("message", err.Error()))
}
}
func shutdown(c *cfg) {
for _, closer := range c.closers {
closer()
}
c.log.Debug("waiting for all processes to stop")
c.wg.Wait()
}
func (c *cfg) onShutdown(f func()) {
c.closers = append(c.closers, f)
}

View file

@ -0,0 +1,47 @@
package main
import (
"context"
metricsconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/metrics"
httputil "github.com/TrueCloudLab/frostfs-node/pkg/util/http"
"github.com/prometheus/client_golang/prometheus/promhttp"
"go.uber.org/zap"
)
func initMetrics(c *cfg) {
if !metricsconfig.Enabled(c.appCfg) {
c.log.Info("prometheus is disabled")
return
}
var prm httputil.Prm
prm.Address = metricsconfig.Address(c.appCfg)
prm.Handler = promhttp.Handler()
srv := httputil.New(prm,
httputil.WithShutdownTimeout(
metricsconfig.ShutdownTimeout(c.appCfg),
),
)
c.workers = append(c.workers, newWorkerFromFunc(func(context.Context) {
runAndLog(c, "metrics", false, func(c *cfg) {
fatalOnErr(srv.Serve())
})
}))
c.closers = append(c.closers, func() {
c.log.Debug("shutting down metrics service")
err := srv.Shutdown()
if err != nil {
c.log.Debug("could not shutdown metrics server",
zap.String("error", err.Error()),
)
}
c.log.Debug("metrics service has been stopped")
})
}

305
cmd/frostfs-node/morph.go Normal file
View file

@ -0,0 +1,305 @@
package main
import (
"context"
"errors"
"fmt"
"time"
morphconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/morph"
"github.com/TrueCloudLab/frostfs-node/pkg/core/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/client"
nmClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/event"
netmapEvent "github.com/TrueCloudLab/frostfs-node/pkg/morph/event/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/subscriber"
"github.com/TrueCloudLab/frostfs-node/pkg/util/rand"
"github.com/nspcc-dev/neo-go/pkg/core/block"
"github.com/nspcc-dev/neo-go/pkg/core/state"
"github.com/nspcc-dev/neo-go/pkg/util"
"go.uber.org/zap"
)
const (
newEpochNotification = "NewEpoch"
// notaryDepositExtraBlocks is the amount of extra blocks to overlap two deposits,
// we do that to make sure that there won't be any blocks without deposited
// assets in a notary contract; make sure it is bigger than any extra rounding
// value in a notary client.
notaryDepositExtraBlocks = 300
// amount of tries(blocks) before notary deposit timeout.
notaryDepositRetriesAmount
)
func initMorphComponents(c *cfg) {
var err error
addresses := morphconfig.RPCEndpoint(c.appCfg)
// Morph client stable-sorts endpoints by priority. Shuffle here to randomize
// order of endpoints with the same priority.
rand.Shuffle(len(addresses), func(i, j int) {
addresses[i], addresses[j] = addresses[j], addresses[i]
})
cli, err := client.New(c.key,
client.WithDialTimeout(morphconfig.DialTimeout(c.appCfg)),
client.WithLogger(c.log),
client.WithEndpoints(addresses...),
client.WithConnLostCallback(func() {
c.internalErr <- errors.New("morph connection has been lost")
}),
client.WithSwitchInterval(morphconfig.SwitchInterval(c.appCfg)),
)
if err != nil {
c.log.Info("failed to create neo RPC client",
zap.Any("endpoints", addresses),
zap.String("error", err.Error()),
)
fatalOnErr(err)
}
c.onShutdown(cli.Close)
if err := cli.SetGroupSignerScope(); err != nil {
c.log.Info("failed to set group signer scope, continue with Global", zap.Error(err))
}
c.cfgMorph.client = cli
c.cfgMorph.notaryEnabled = cli.ProbeNotary()
lookupScriptHashesInNNS(c) // smart contract auto negotiation
if c.cfgMorph.notaryEnabled {
err = c.cfgMorph.client.EnableNotarySupport(
client.WithProxyContract(
c.cfgMorph.proxyScriptHash,
),
)
fatalOnErr(err)
}
c.log.Info("notary support",
zap.Bool("sidechain_enabled", c.cfgMorph.notaryEnabled),
)
wrap, err := nmClient.NewFromMorph(c.cfgMorph.client, c.cfgNetmap.scriptHash, 0, nmClient.TryNotary())
fatalOnErr(err)
var netmapSource netmap.Source
c.cfgMorph.cacheTTL = morphconfig.CacheTTL(c.appCfg)
if c.cfgMorph.cacheTTL == 0 {
msPerBlock, err := c.cfgMorph.client.MsPerBlock()
fatalOnErr(err)
c.cfgMorph.cacheTTL = time.Duration(msPerBlock) * time.Millisecond
c.log.Debug("morph.cache_ttl fetched from network", zap.Duration("value", c.cfgMorph.cacheTTL))
}
if c.cfgMorph.cacheTTL < 0 {
netmapSource = wrap
} else {
// use RPC node as source of netmap (with caching)
netmapSource = newCachedNetmapStorage(c.cfgNetmap.state, wrap)
}
c.netMapSource = netmapSource
c.cfgNetmap.wrapper = wrap
}
func makeAndWaitNotaryDeposit(c *cfg) {
// skip notary deposit in non-notary environments
if !c.cfgMorph.notaryEnabled {
return
}
tx, err := makeNotaryDeposit(c)
fatalOnErr(err)
err = waitNotaryDeposit(c, tx)
fatalOnErr(err)
}
func makeNotaryDeposit(c *cfg) (util.Uint256, error) {
const (
// gasMultiplier defines how many times more the notary
// balance must be compared to the GAS balance of the node:
// notaryBalance = GASBalance * gasMultiplier
gasMultiplier = 3
// gasDivisor defines what part of GAS balance (1/gasDivisor)
// should be transferred to the notary service
gasDivisor = 2
)
depositAmount, err := client.CalculateNotaryDepositAmount(c.cfgMorph.client, gasMultiplier, gasDivisor)
if err != nil {
return util.Uint256{}, fmt.Errorf("could not calculate notary deposit: %w", err)
}
return c.cfgMorph.client.DepositEndlessNotary(depositAmount)
}
var (
errNotaryDepositFail = errors.New("notary deposit tx has faulted")
errNotaryDepositTimeout = errors.New("notary deposit tx has not appeared in the network")
)
func waitNotaryDeposit(c *cfg, tx util.Uint256) error {
for i := 0; i < notaryDepositRetriesAmount; i++ {
select {
case <-c.ctx.Done():
return c.ctx.Err()
default:
}
ok, err := c.cfgMorph.client.TxHalt(tx)
if err == nil {
if ok {
return nil
}
return errNotaryDepositFail
}
err = c.cfgMorph.client.Wait(c.ctx, 1)
if err != nil {
return fmt.Errorf("could not wait for one block in chain: %w", err)
}
}
return errNotaryDepositTimeout
}
func listenMorphNotifications(c *cfg) {
// listenerPoolCap is a capacity of a
// worker pool inside the listener. It
// is used to prevent blocking in neo-go:
// the client cannot make RPC requests if
// the notification channel is not being
// read by another goroutine.
const listenerPoolCap = 10
var (
err error
subs subscriber.Subscriber
)
fromSideChainBlock, err := c.persistate.UInt32(persistateSideChainLastBlockKey)
if err != nil {
fromSideChainBlock = 0
c.log.Warn("can't get last processed side chain block number", zap.String("error", err.Error()))
}
subs, err = subscriber.New(c.ctx, &subscriber.Params{
Log: c.log,
StartFromBlock: fromSideChainBlock,
Client: c.cfgMorph.client,
})
fatalOnErr(err)
lis, err := event.NewListener(event.ListenerParams{
Logger: c.log,
Subscriber: subs,
WorkerPoolCapacity: listenerPoolCap,
})
fatalOnErr(err)
c.workers = append(c.workers, newWorkerFromFunc(func(ctx context.Context) {
runAndLog(c, "morph notification", false, func(c *cfg) {
lis.ListenWithError(ctx, c.internalErr)
})
}))
setNetmapNotificationParser(c, newEpochNotification, func(src *state.ContainedNotificationEvent) (event.Event, error) {
res, err := netmapEvent.ParseNewEpoch(src)
if err == nil {
c.log.Info("new epoch event from sidechain",
zap.Uint64("number", res.(netmapEvent.NewEpoch).EpochNumber()),
)
}
return res, err
})
registerNotificationHandlers(c.cfgNetmap.scriptHash, lis, c.cfgNetmap.parsers, c.cfgNetmap.subscribers)
registerNotificationHandlers(c.cfgContainer.scriptHash, lis, c.cfgContainer.parsers, c.cfgContainer.subscribers)
registerBlockHandler(lis, func(block *block.Block) {
c.log.Debug("new block", zap.Uint32("index", block.Index))
err = c.persistate.SetUInt32(persistateSideChainLastBlockKey, block.Index)
if err != nil {
c.log.Warn("can't update persistent state",
zap.String("chain", "side"),
zap.Uint32("block_index", block.Index))
}
tickBlockTimers(c)
})
}
func registerNotificationHandlers(scHash util.Uint160, lis event.Listener, parsers map[event.Type]event.NotificationParser,
subs map[event.Type][]event.Handler) {
for typ, handlers := range subs {
pi := event.NotificationParserInfo{}
pi.SetType(typ)
pi.SetScriptHash(scHash)
p, ok := parsers[typ]
if !ok {
panic(fmt.Sprintf("missing parser for event %s", typ))
}
pi.SetParser(p)
lis.SetNotificationParser(pi)
for _, h := range handlers {
hi := event.NotificationHandlerInfo{}
hi.SetType(typ)
hi.SetScriptHash(scHash)
hi.SetHandler(h)
lis.RegisterNotificationHandler(hi)
}
}
}
func registerBlockHandler(lis event.Listener, handler event.BlockHandler) {
lis.RegisterBlockHandler(handler)
}
// lookupScriptHashesInNNS looks up for contract script hashes in NNS contract of side
// chain if they were not specified in config file.
func lookupScriptHashesInNNS(c *cfg) {
var (
err error
emptyHash = util.Uint160{}
targets = [...]struct {
h *util.Uint160
nnsName string
}{
{&c.cfgNetmap.scriptHash, client.NNSNetmapContractName},
{&c.cfgAccounting.scriptHash, client.NNSBalanceContractName},
{&c.cfgContainer.scriptHash, client.NNSContainerContractName},
{&c.cfgReputation.scriptHash, client.NNSReputationContractName},
{&c.cfgMorph.proxyScriptHash, client.NNSProxyContractName},
}
)
for _, t := range targets {
if t.nnsName == client.NNSProxyContractName && !c.cfgMorph.notaryEnabled {
continue // ignore proxy contract if notary disabled
}
if emptyHash.Equals(*t.h) {
*t.h, err = c.cfgMorph.client.NNSContractAddress(t.nnsName)
fatalOnErrDetails(fmt.Sprintf("can't resolve %s in NNS", t.nnsName), err)
}
}
}

467
cmd/frostfs-node/netmap.go Normal file
View file

@ -0,0 +1,467 @@
package main
import (
"bytes"
"errors"
"fmt"
netmapGRPC "github.com/TrueCloudLab/frostfs-api-go/v2/netmap/grpc"
nodeconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/node"
"github.com/TrueCloudLab/frostfs-node/pkg/core/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/metrics"
nmClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/event"
netmapEvent "github.com/TrueCloudLab/frostfs-node/pkg/morph/event/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/network"
netmapTransportGRPC "github.com/TrueCloudLab/frostfs-node/pkg/network/transport/netmap/grpc"
"github.com/TrueCloudLab/frostfs-node/pkg/services/control"
netmapService "github.com/TrueCloudLab/frostfs-node/pkg/services/netmap"
netmapSDK "github.com/TrueCloudLab/frostfs-sdk-go/netmap"
subnetid "github.com/TrueCloudLab/frostfs-sdk-go/subnet/id"
"github.com/TrueCloudLab/frostfs-sdk-go/version"
"go.uber.org/atomic"
"go.uber.org/zap"
)
// primary solution of local network state dump.
type networkState struct {
epoch *atomic.Uint64
controlNetStatus atomic.Value // control.NetmapStatus
nodeInfo atomic.Value // *netmapSDK.NodeInfo
metrics *metrics.NodeMetrics
}
func newNetworkState() *networkState {
var nmStatus atomic.Value
nmStatus.Store(control.NetmapStatus_STATUS_UNDEFINED)
return &networkState{
epoch: atomic.NewUint64(0),
controlNetStatus: nmStatus,
}
}
func (s *networkState) CurrentEpoch() uint64 {
return s.epoch.Load()
}
func (s *networkState) setCurrentEpoch(v uint64) {
s.epoch.Store(v)
if s.metrics != nil {
s.metrics.SetEpoch(v)
}
}
func (s *networkState) setNodeInfo(ni *netmapSDK.NodeInfo) {
ctrlNetSt := control.NetmapStatus_STATUS_UNDEFINED
if ni != nil {
s.nodeInfo.Store(*ni)
switch {
case ni.IsOnline():
ctrlNetSt = control.NetmapStatus_ONLINE
case ni.IsOffline():
ctrlNetSt = control.NetmapStatus_OFFLINE
case ni.IsMaintenance():
ctrlNetSt = control.NetmapStatus_MAINTENANCE
}
} else {
ctrlNetSt = control.NetmapStatus_OFFLINE
niRaw := s.nodeInfo.Load()
if niRaw != nil {
niOld := niRaw.(netmapSDK.NodeInfo)
// nil ni means that the node is not included
// in the netmap
niOld.SetOffline()
s.nodeInfo.Store(niOld)
}
}
s.setControlNetmapStatus(ctrlNetSt)
}
// sets the current node state to the given value. Subsequent cfg.bootstrap
// calls will process this value to decide what status node should set in the
// network.
func (s *networkState) setControlNetmapStatus(st control.NetmapStatus) {
s.controlNetStatus.Store(st)
}
func (s *networkState) controlNetmapStatus() (res control.NetmapStatus) {
return s.controlNetStatus.Load().(control.NetmapStatus)
}
func (s *networkState) getNodeInfo() (res netmapSDK.NodeInfo, ok bool) {
v := s.nodeInfo.Load()
if v != nil {
res, ok = v.(netmapSDK.NodeInfo)
if !ok {
panic(fmt.Sprintf("unexpected value in atomic node info state: %T", v))
}
}
return
}
func nodeKeyFromNetmap(c *cfg) []byte {
ni, ok := c.cfgNetmap.state.getNodeInfo()
if ok {
return ni.PublicKey()
}
return nil
}
func (c *cfg) iterateNetworkAddresses(f func(string) bool) {
ni, ok := c.cfgNetmap.state.getNodeInfo()
if ok {
ni.IterateNetworkEndpoints(f)
}
}
func (c *cfg) addressNum() int {
ni, ok := c.cfgNetmap.state.getNodeInfo()
if ok {
return ni.NumberOfNetworkEndpoints()
}
return 0
}
func initNetmapService(c *cfg) {
network.WriteToNodeInfo(c.localAddr, &c.cfgNodeInfo.localInfo)
c.cfgNodeInfo.localInfo.SetPublicKey(c.key.PublicKey().Bytes())
parseAttributes(c)
c.cfgNodeInfo.localInfo.SetOffline()
readSubnetCfg(c)
if c.cfgMorph.client == nil {
initMorphComponents(c)
}
initNetmapState(c)
server := netmapTransportGRPC.New(
netmapService.NewSignService(
&c.key.PrivateKey,
netmapService.NewResponseService(
netmapService.NewExecutionService(
c,
c.apiVersion,
&netInfo{
netState: c.cfgNetmap.state,
magic: c.cfgMorph.client,
morphClientNetMap: c.cfgNetmap.wrapper,
msPerBlockRdr: c.cfgMorph.client.MsPerBlock,
},
),
c.respSvc,
),
),
)
for _, srv := range c.cfgGRPC.servers {
netmapGRPC.RegisterNetmapServiceServer(srv, server)
}
addNewEpochNotificationHandler(c, func(ev event.Event) {
c.cfgNetmap.state.setCurrentEpoch(ev.(netmapEvent.NewEpoch).EpochNumber())
})
addNewEpochAsyncNotificationHandler(c, func(ev event.Event) {
if !c.needBootstrap() || c.cfgNetmap.reBoostrapTurnedOff.Load() { // fixes #470
return
}
n := ev.(netmapEvent.NewEpoch).EpochNumber()
const reBootstrapInterval = 2
if (n-c.cfgNetmap.startEpoch)%reBootstrapInterval == 0 {
err := c.bootstrap()
if err != nil {
c.log.Warn("can't send re-bootstrap tx", zap.Error(err))
}
}
})
addNewEpochAsyncNotificationHandler(c, func(ev event.Event) {
e := ev.(netmapEvent.NewEpoch).EpochNumber()
ni, err := c.netmapLocalNodeState(e)
if err != nil {
c.log.Error("could not update node state on new epoch",
zap.Uint64("epoch", e),
zap.String("error", err.Error()),
)
return
}
c.handleLocalNodeInfo(ni)
})
if c.cfgMorph.notaryEnabled {
addNewEpochAsyncNotificationHandler(c, func(ev event.Event) {
_, err := makeNotaryDeposit(c)
if err != nil {
c.log.Error("could not make notary deposit",
zap.String("error", err.Error()),
)
}
})
}
}
func readSubnetCfg(c *cfg) {
var subnetCfg nodeconfig.SubnetConfig
subnetCfg.Init(*c.appCfg)
var (
id subnetid.ID
err error
)
subnetCfg.IterateSubnets(func(idTxt string) {
err = id.DecodeString(idTxt)
fatalOnErrDetails("parse subnet entry", err)
c.cfgNodeInfo.localInfo.EnterSubnet(id)
})
if subnetCfg.ExitZero() {
subnetid.MakeZero(&id)
c.cfgNodeInfo.localInfo.ExitSubnet(id)
}
}
// bootstrapNode adds current node to the Network map.
// Must be called after initNetmapService.
func bootstrapNode(c *cfg) {
if c.needBootstrap() {
err := c.bootstrap()
fatalOnErrDetails("bootstrap error", err)
}
}
func addNetmapNotificationHandler(c *cfg, sTyp string, h event.Handler) {
typ := event.TypeFromString(sTyp)
if c.cfgNetmap.subscribers == nil {
c.cfgNetmap.subscribers = make(map[event.Type][]event.Handler, 1)
}
c.cfgNetmap.subscribers[typ] = append(c.cfgNetmap.subscribers[typ], h)
}
func setNetmapNotificationParser(c *cfg, sTyp string, p event.NotificationParser) {
typ := event.TypeFromString(sTyp)
if c.cfgNetmap.parsers == nil {
c.cfgNetmap.parsers = make(map[event.Type]event.NotificationParser, 1)
}
c.cfgNetmap.parsers[typ] = p
}
// initNetmapState inits current Network map state.
// Must be called after Morph components initialization.
func initNetmapState(c *cfg) {
epoch, err := c.cfgNetmap.wrapper.Epoch()
fatalOnErrDetails("could not initialize current epoch number", err)
ni, err := c.netmapLocalNodeState(epoch)
fatalOnErrDetails("could not init network state", err)
stateWord := "undefined"
if ni != nil {
switch {
case ni.IsOnline():
stateWord = "online"
case ni.IsOffline():
stateWord = "offline"
}
}
c.log.Info("initial network state",
zap.Uint64("epoch", epoch),
zap.String("state", stateWord),
)
c.cfgNetmap.state.setCurrentEpoch(epoch)
c.cfgNetmap.startEpoch = epoch
c.handleLocalNodeInfo(ni)
}
func (c *cfg) netmapLocalNodeState(epoch uint64) (*netmapSDK.NodeInfo, error) {
// calculate current network state
nm, err := c.cfgNetmap.wrapper.GetNetMapByEpoch(epoch)
if err != nil {
return nil, err
}
c.netMap.Store(*nm)
nmNodes := nm.Nodes()
for i := range nmNodes {
if bytes.Equal(nmNodes[i].PublicKey(), c.binPublicKey) {
return &nmNodes[i], nil
}
}
return nil, nil
}
// addNewEpochNotificationHandler adds handler that will be executed synchronously.
func addNewEpochNotificationHandler(c *cfg, h event.Handler) {
addNetmapNotificationHandler(c, newEpochNotification, h)
}
// addNewEpochAsyncNotificationHandler adds handler that will be executed asynchronously via netmap workerPool.
func addNewEpochAsyncNotificationHandler(c *cfg, h event.Handler) {
addNetmapNotificationHandler(
c,
newEpochNotification,
event.WorkerPoolHandler(
c.cfgNetmap.workerPool,
h,
c.log,
),
)
}
var errRelayBootstrap = errors.New("setting netmap status is forbidden in relay mode")
func (c *cfg) SetNetmapStatus(st control.NetmapStatus) error {
switch st {
default:
return fmt.Errorf("unsupported status %v", st)
case control.NetmapStatus_MAINTENANCE:
return c.setMaintenanceStatus(false)
case control.NetmapStatus_ONLINE, control.NetmapStatus_OFFLINE:
}
c.stopMaintenance()
if !c.needBootstrap() {
return errRelayBootstrap
}
if st == control.NetmapStatus_ONLINE {
c.cfgNetmap.reBoostrapTurnedOff.Store(false)
return bootstrapOnline(c)
}
c.cfgNetmap.reBoostrapTurnedOff.Store(true)
return c.updateNetMapState(func(*nmClient.UpdatePeerPrm) {})
}
func (c *cfg) ForceMaintenance() error {
return c.setMaintenanceStatus(true)
}
func (c *cfg) setMaintenanceStatus(force bool) error {
netSettings, err := c.cfgNetmap.wrapper.ReadNetworkConfiguration()
if err != nil {
err = fmt.Errorf("read network settings to check maintenance allowance: %w", err)
} else if !netSettings.MaintenanceModeAllowed {
err = errors.New("maintenance mode is not allowed by the network")
}
if err == nil || force {
c.startMaintenance()
if err == nil {
err = c.updateNetMapState((*nmClient.UpdatePeerPrm).SetMaintenance)
}
if err != nil {
return fmt.Errorf("local maintenance is started, but state is not updated in the network: %w", err)
}
}
return err
}
// calls UpdatePeerState operation of Netmap contract's client for the local node.
// State setter is used to specify node state to switch to.
func (c *cfg) updateNetMapState(stateSetter func(*nmClient.UpdatePeerPrm)) error {
var prm nmClient.UpdatePeerPrm
prm.SetKey(c.key.PublicKey().Bytes())
stateSetter(&prm)
return c.cfgNetmap.wrapper.UpdatePeerState(prm)
}
type netInfo struct {
netState netmap.State
magic interface {
MagicNumber() (uint64, error)
}
morphClientNetMap *nmClient.Client
msPerBlockRdr func() (int64, error)
}
func (n *netInfo) Dump(ver version.Version) (*netmapSDK.NetworkInfo, error) {
magic, err := n.magic.MagicNumber()
if err != nil {
return nil, err
}
var ni netmapSDK.NetworkInfo
ni.SetCurrentEpoch(n.netState.CurrentEpoch())
ni.SetMagicNumber(magic)
netInfoMorph, err := n.morphClientNetMap.ReadNetworkConfiguration()
if err != nil {
return nil, fmt.Errorf("read network configuration using netmap contract client: %w", err)
}
if mjr := ver.Major(); mjr > 2 || mjr == 2 && ver.Minor() > 9 {
msPerBlock, err := n.msPerBlockRdr()
if err != nil {
return nil, fmt.Errorf("ms per block: %w", err)
}
ni.SetMsPerBlock(msPerBlock)
ni.SetMaxObjectSize(netInfoMorph.MaxObjectSize)
ni.SetStoragePrice(netInfoMorph.StoragePrice)
ni.SetAuditFee(netInfoMorph.AuditFee)
ni.SetEpochDuration(netInfoMorph.EpochDuration)
ni.SetContainerFee(netInfoMorph.ContainerFee)
ni.SetNamedContainerFee(netInfoMorph.ContainerAliasFee)
ni.SetNumberOfEigenTrustIterations(netInfoMorph.EigenTrustIterations)
ni.SetEigenTrustAlpha(netInfoMorph.EigenTrustAlpha)
ni.SetIRCandidateFee(netInfoMorph.IRCandidateFee)
ni.SetWithdrawalFee(netInfoMorph.WithdrawalFee)
if netInfoMorph.HomomorphicHashingDisabled {
ni.DisableHomomorphicHashing()
}
if netInfoMorph.MaintenanceModeAllowed {
ni.AllowMaintenanceMode()
}
for i := range netInfoMorph.Raw {
ni.SetRawNetworkParameter(netInfoMorph.Raw[i].Name, netInfoMorph.Raw[i].Value)
}
}
return &ni, nil
}

View file

@ -0,0 +1,168 @@
package main
import (
"encoding/hex"
"fmt"
nodeconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/node"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/engine"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/event"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/event/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/services/notificator"
"github.com/TrueCloudLab/frostfs-node/pkg/services/notificator/nats"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
objectSDK "github.com/TrueCloudLab/frostfs-sdk-go/object"
oid "github.com/TrueCloudLab/frostfs-sdk-go/object/id"
"go.uber.org/zap"
)
type notificationSource struct {
e *engine.StorageEngine
l *logger.Logger
defaultTopic string
}
func (n *notificationSource) Iterate(epoch uint64, handler func(topic string, addr oid.Address)) {
log := n.l.With(zap.Uint64("epoch", epoch))
listRes, err := n.e.ListContainers(engine.ListContainersPrm{})
if err != nil {
log.Error("notificator: could not list containers", zap.Error(err))
return
}
filters := objectSDK.NewSearchFilters()
filters.AddNotificationEpochFilter(epoch)
var selectPrm engine.SelectPrm
selectPrm.WithFilters(filters)
for _, c := range listRes.Containers() {
selectPrm.WithContainerID(c)
selectRes, err := n.e.Select(selectPrm)
if err != nil {
log.Error("notificator: could not select objects from container",
zap.Stringer("cid", c),
zap.Error(err),
)
continue
}
for _, a := range selectRes.AddressList() {
err = n.processAddress(a, handler)
if err != nil {
log.Error("notificator: could not process object",
zap.Stringer("address", a),
zap.Error(err),
)
continue
}
}
}
log.Debug("notificator: finished processing object notifications")
}
func (n *notificationSource) processAddress(
a oid.Address,
h func(topic string, addr oid.Address),
) error {
var prm engine.HeadPrm
prm.WithAddress(a)
res, err := n.e.Head(prm)
if err != nil {
return err
}
ni, err := res.Header().NotificationInfo()
if err != nil {
return fmt.Errorf("could not retrieve notification topic from object: %w", err)
}
topic := ni.Topic()
if topic == "" {
topic = n.defaultTopic
}
h(topic, a)
return nil
}
type notificationWriter struct {
l *logger.Logger
w *nats.Writer
}
func (n notificationWriter) Notify(topic string, address oid.Address) {
if err := n.w.Notify(topic, address); err != nil {
n.l.Warn("could not write object notification",
zap.Stringer("address", address),
zap.String("topic", topic),
zap.Error(err),
)
}
}
func initNotifications(c *cfg) {
if nodeconfig.Notification(c.appCfg).Enabled() {
topic := nodeconfig.Notification(c.appCfg).DefaultTopic()
pubKey := hex.EncodeToString(c.cfgNodeInfo.localInfo.PublicKey())
if topic == "" {
topic = pubKey
}
natsSvc := nats.New(
nats.WithConnectionName("NeoFS Storage Node: "+pubKey), // connection name is used in the server side logs
nats.WithTimeout(nodeconfig.Notification(c.appCfg).Timeout()),
nats.WithClientCert(
nodeconfig.Notification(c.appCfg).CertPath(),
nodeconfig.Notification(c.appCfg).KeyPath(),
),
nats.WithRootCA(nodeconfig.Notification(c.appCfg).CAPath()),
nats.WithLogger(c.log),
)
c.cfgNotifications = cfgNotifications{
enabled: true,
nw: notificationWriter{
l: c.log,
w: natsSvc,
},
defaultTopic: topic,
}
n := notificator.New(new(notificator.Prm).
SetLogger(c.log).
SetNotificationSource(
&notificationSource{
e: c.cfgObject.cfgLocalStorage.localStorage,
l: c.log,
defaultTopic: topic,
}).
SetWriter(c.cfgNotifications.nw),
)
addNewEpochAsyncNotificationHandler(c, func(e event.Event) {
ev := e.(netmap.NewEpoch)
n.ProcessEpoch(ev.EpochNumber())
})
}
}
func connectNats(c *cfg) {
if !c.cfgNotifications.enabled {
return
}
endpoint := nodeconfig.Notification(c.appCfg).Endpoint()
err := c.cfgNotifications.nw.w.Connect(c.ctx, endpoint)
if err != nil {
panic(fmt.Sprintf("could not connect to a nats endpoint %s: %v", endpoint, err))
}
}

601
cmd/frostfs-node/object.go Normal file
View file

@ -0,0 +1,601 @@
package main
import (
"bytes"
"context"
"errors"
"fmt"
"github.com/TrueCloudLab/frostfs-api-go/v2/object"
objectGRPC "github.com/TrueCloudLab/frostfs-api-go/v2/object/grpc"
policerconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/policer"
replicatorconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/replicator"
coreclient "github.com/TrueCloudLab/frostfs-node/pkg/core/client"
containercore "github.com/TrueCloudLab/frostfs-node/pkg/core/container"
"github.com/TrueCloudLab/frostfs-node/pkg/core/netmap"
objectCore "github.com/TrueCloudLab/frostfs-node/pkg/core/object"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/engine"
morphClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client"
cntClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client/container"
nmClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client/netmap"
objectTransportGRPC "github.com/TrueCloudLab/frostfs-node/pkg/network/transport/object/grpc"
objectService "github.com/TrueCloudLab/frostfs-node/pkg/services/object"
"github.com/TrueCloudLab/frostfs-node/pkg/services/object/acl"
v2 "github.com/TrueCloudLab/frostfs-node/pkg/services/object/acl/v2"
deletesvc "github.com/TrueCloudLab/frostfs-node/pkg/services/object/delete"
deletesvcV2 "github.com/TrueCloudLab/frostfs-node/pkg/services/object/delete/v2"
getsvc "github.com/TrueCloudLab/frostfs-node/pkg/services/object/get"
getsvcV2 "github.com/TrueCloudLab/frostfs-node/pkg/services/object/get/v2"
headsvc "github.com/TrueCloudLab/frostfs-node/pkg/services/object/head"
putsvc "github.com/TrueCloudLab/frostfs-node/pkg/services/object/put"
putsvcV2 "github.com/TrueCloudLab/frostfs-node/pkg/services/object/put/v2"
searchsvc "github.com/TrueCloudLab/frostfs-node/pkg/services/object/search"
searchsvcV2 "github.com/TrueCloudLab/frostfs-node/pkg/services/object/search/v2"
"github.com/TrueCloudLab/frostfs-node/pkg/services/object/util"
"github.com/TrueCloudLab/frostfs-node/pkg/services/object_manager/placement"
"github.com/TrueCloudLab/frostfs-node/pkg/services/policer"
"github.com/TrueCloudLab/frostfs-node/pkg/services/replicator"
truststorage "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/local/storage"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
"github.com/TrueCloudLab/frostfs-sdk-go/client"
apistatus "github.com/TrueCloudLab/frostfs-sdk-go/client/status"
cid "github.com/TrueCloudLab/frostfs-sdk-go/container/id"
eaclSDK "github.com/TrueCloudLab/frostfs-sdk-go/eacl"
objectSDK "github.com/TrueCloudLab/frostfs-sdk-go/object"
oid "github.com/TrueCloudLab/frostfs-sdk-go/object/id"
apireputation "github.com/TrueCloudLab/frostfs-sdk-go/reputation"
"github.com/TrueCloudLab/frostfs-sdk-go/user"
"go.uber.org/zap"
)
type objectSvc struct {
put *putsvcV2.Service
search *searchsvcV2.Service
get *getsvcV2.Service
delete *deletesvcV2.Service
}
func (c *cfg) MaxObjectSize() uint64 {
sz, err := c.cfgNetmap.wrapper.MaxObjectSize()
if err != nil {
c.log.Error("could not get max object size value",
zap.String("error", err.Error()),
)
}
return sz
}
func (s *objectSvc) Put(ctx context.Context) (objectService.PutObjectStream, error) {
return s.put.Put(ctx)
}
func (s *objectSvc) Head(ctx context.Context, req *object.HeadRequest) (*object.HeadResponse, error) {
return s.get.Head(ctx, req)
}
func (s *objectSvc) Search(req *object.SearchRequest, stream objectService.SearchStream) error {
return s.search.Search(req, stream)
}
func (s *objectSvc) Get(req *object.GetRequest, stream objectService.GetObjectStream) error {
return s.get.Get(req, stream)
}
func (s *objectSvc) Delete(ctx context.Context, req *object.DeleteRequest) (*object.DeleteResponse, error) {
return s.delete.Delete(ctx, req)
}
func (s *objectSvc) GetRange(req *object.GetRangeRequest, stream objectService.GetObjectRangeStream) error {
return s.get.GetRange(req, stream)
}
func (s *objectSvc) GetRangeHash(ctx context.Context, req *object.GetRangeHashRequest) (*object.GetRangeHashResponse, error) {
return s.get.GetRangeHash(ctx, req)
}
type delNetInfo struct {
netmap.State
tsLifetime uint64
cfg *cfg
}
func (i *delNetInfo) TombstoneLifetime() (uint64, error) {
return i.tsLifetime, nil
}
// returns node owner ID calculated from configured private key.
//
// Implements method needed for Object.Delete service.
func (i *delNetInfo) LocalNodeID() user.ID {
return i.cfg.ownerIDFromKey
}
type innerRingFetcherWithNotary struct {
sidechain *morphClient.Client
}
func (fn *innerRingFetcherWithNotary) InnerRingKeys() ([][]byte, error) {
keys, err := fn.sidechain.NeoFSAlphabetList()
if err != nil {
return nil, fmt.Errorf("can't get inner ring keys from alphabet role: %w", err)
}
result := make([][]byte, 0, len(keys))
for i := range keys {
result = append(result, keys[i].Bytes())
}
return result, nil
}
type innerRingFetcherWithoutNotary struct {
nm *nmClient.Client
}
func (f *innerRingFetcherWithoutNotary) InnerRingKeys() ([][]byte, error) {
keys, err := f.nm.GetInnerRingList()
if err != nil {
return nil, fmt.Errorf("can't get inner ring keys from netmap contract: %w", err)
}
result := make([][]byte, 0, len(keys))
for i := range keys {
result = append(result, keys[i].Bytes())
}
return result, nil
}
type coreClientConstructor reputationClientConstructor
func (x *coreClientConstructor) Get(info coreclient.NodeInfo) (coreclient.MultiAddressClient, error) {
c, err := (*reputationClientConstructor)(x).Get(info)
if err != nil {
return nil, err
}
return c.(coreclient.MultiAddressClient), nil
}
func initObjectService(c *cfg) {
ls := c.cfgObject.cfgLocalStorage.localStorage
keyStorage := util.NewKeyStorage(&c.key.PrivateKey, c.privateTokenStore, c.cfgNetmap.state)
clientConstructor := &reputationClientConstructor{
log: c.log,
nmSrc: c.netMapSource,
netState: c.cfgNetmap.state,
trustStorage: c.cfgReputation.localTrustStorage,
basicConstructor: c.bgClientCache,
}
coreConstructor := &coreClientConstructor{
log: c.log,
nmSrc: c.netMapSource,
netState: c.cfgNetmap.state,
trustStorage: c.cfgReputation.localTrustStorage,
basicConstructor: c.clientCache,
}
var irFetcher v2.InnerRingFetcher
if c.cfgMorph.client.ProbeNotary() {
irFetcher = &innerRingFetcherWithNotary{
sidechain: c.cfgMorph.client,
}
} else {
irFetcher = &innerRingFetcherWithoutNotary{
nm: c.cfgNetmap.wrapper,
}
}
c.replicator = replicator.New(
replicator.WithLogger(c.log),
replicator.WithPutTimeout(
replicatorconfig.PutTimeout(c.appCfg),
),
replicator.WithLocalStorage(ls),
replicator.WithRemoteSender(
putsvc.NewRemoteSender(keyStorage, coreConstructor),
),
)
pol := policer.New(
policer.WithLogger(c.log),
policer.WithLocalStorage(ls),
policer.WithContainerSource(c.cfgObject.cnrSource),
policer.WithPlacementBuilder(
placement.NewNetworkMapSourceBuilder(c.netMapSource),
),
policer.WithRemoteHeader(
headsvc.NewRemoteHeader(keyStorage, clientConstructor),
),
policer.WithNetmapKeys(c),
policer.WithHeadTimeout(
policerconfig.HeadTimeout(c.appCfg),
),
policer.WithReplicator(c.replicator),
policer.WithRedundantCopyCallback(func(addr oid.Address) {
var inhumePrm engine.InhumePrm
inhumePrm.MarkAsGarbage(addr)
_, err := ls.Inhume(inhumePrm)
if err != nil {
c.log.Warn("could not inhume mark redundant copy as garbage",
zap.String("error", err.Error()),
)
}
}),
policer.WithMaxCapacity(c.cfgObject.pool.replicatorPoolSize),
policer.WithPool(c.cfgObject.pool.replication),
policer.WithNodeLoader(c),
)
traverseGen := util.NewTraverserGenerator(c.netMapSource, c.cfgObject.cnrSource, c)
c.workers = append(c.workers, pol)
var os putsvc.ObjectStorage = engineWithoutNotifications{
engine: ls,
}
if c.cfgNotifications.enabled {
os = engineWithNotifications{
base: os,
nw: c.cfgNotifications.nw,
ns: c.cfgNetmap.state,
defaultTopic: c.cfgNotifications.defaultTopic,
}
}
sPut := putsvc.NewService(
putsvc.WithKeyStorage(keyStorage),
putsvc.WithClientConstructor(coreConstructor),
putsvc.WithMaxSizeSource(newCachedMaxObjectSizeSource(c)),
putsvc.WithObjectStorage(os),
putsvc.WithContainerSource(c.cfgObject.cnrSource),
putsvc.WithNetworkMapSource(c.netMapSource),
putsvc.WithNetmapKeys(c),
putsvc.WithNetworkState(c.cfgNetmap.state),
putsvc.WithWorkerPools(c.cfgObject.pool.putRemote),
putsvc.WithLogger(c.log),
)
sPutV2 := putsvcV2.NewService(
putsvcV2.WithInternalService(sPut),
putsvcV2.WithKeyStorage(keyStorage),
)
sSearch := searchsvc.New(
searchsvc.WithLogger(c.log),
searchsvc.WithLocalStorageEngine(ls),
searchsvc.WithClientConstructor(coreConstructor),
searchsvc.WithTraverserGenerator(
traverseGen.WithTraverseOptions(
placement.WithoutSuccessTracking(),
),
),
searchsvc.WithNetMapSource(c.netMapSource),
searchsvc.WithKeyStorage(keyStorage),
)
sSearchV2 := searchsvcV2.NewService(
searchsvcV2.WithInternalService(sSearch),
searchsvcV2.WithKeyStorage(keyStorage),
)
sGet := getsvc.New(
getsvc.WithLogger(c.log),
getsvc.WithLocalStorageEngine(ls),
getsvc.WithClientConstructor(coreConstructor),
getsvc.WithTraverserGenerator(
traverseGen.WithTraverseOptions(
placement.SuccessAfter(1),
),
),
getsvc.WithNetMapSource(c.netMapSource),
getsvc.WithKeyStorage(keyStorage),
)
*c.cfgObject.getSvc = *sGet // need smth better
sGetV2 := getsvcV2.NewService(
getsvcV2.WithInternalService(sGet),
getsvcV2.WithKeyStorage(keyStorage),
)
sDelete := deletesvc.New(
deletesvc.WithLogger(c.log),
deletesvc.WithHeadService(sGet),
deletesvc.WithSearchService(sSearch),
deletesvc.WithPutService(sPut),
deletesvc.WithNetworkInfo(&delNetInfo{
State: c.cfgNetmap.state,
tsLifetime: 5,
cfg: c,
}),
deletesvc.WithKeyStorage(keyStorage),
)
sDeleteV2 := deletesvcV2.NewService(
deletesvcV2.WithInternalService(sDelete),
)
// build service pipeline
// grpc | <metrics> | signature | response | acl | split
splitSvc := objectService.NewTransportSplitter(
c.cfgGRPC.maxChunkSize,
c.cfgGRPC.maxAddrAmount,
&objectSvc{
put: sPutV2,
search: sSearchV2,
get: sGetV2,
delete: sDeleteV2,
},
)
aclSvc := v2.New(
v2.WithLogger(c.log),
v2.WithIRFetcher(newCachedIRFetcher(irFetcher)),
v2.WithNetmapSource(c.netMapSource),
v2.WithContainerSource(
c.cfgObject.cnrSource,
),
v2.WithNextService(splitSvc),
v2.WithEACLChecker(
acl.NewChecker(new(acl.CheckerPrm).
SetNetmapState(c.cfgNetmap.state).
SetEACLSource(c.cfgObject.eaclSource).
SetValidator(eaclSDK.NewValidator()).
SetLocalStorage(ls),
),
),
)
var commonSvc objectService.Common
commonSvc.Init(&c.internals, aclSvc)
respSvc := objectService.NewResponseService(
&commonSvc,
c.respSvc,
)
signSvc := objectService.NewSignService(
&c.key.PrivateKey,
respSvc,
)
var firstSvc objectService.ServiceServer = signSvc
if c.metricsCollector != nil {
firstSvc = objectService.NewMetricCollector(signSvc, c.metricsCollector)
}
server := objectTransportGRPC.New(firstSvc)
for _, srv := range c.cfgGRPC.servers {
objectGRPC.RegisterObjectServiceServer(srv, server)
}
}
type morphEACLFetcher struct {
w *cntClient.Client
}
func (s *morphEACLFetcher) GetEACL(cnr cid.ID) (*containercore.EACL, error) {
eaclInfo, err := s.w.GetEACL(cnr)
if err != nil {
return nil, err
}
binTable, err := eaclInfo.Value.Marshal()
if err != nil {
return nil, fmt.Errorf("marshal eACL table: %w", err)
}
if !eaclInfo.Signature.Verify(binTable) {
// TODO(@cthulhu-rider): #1387 use "const" error
return nil, errors.New("invalid signature of the eACL table")
}
return eaclInfo, nil
}
type reputationClientConstructor struct {
log *logger.Logger
nmSrc netmap.Source
netState netmap.State
trustStorage *truststorage.Storage
basicConstructor interface {
Get(coreclient.NodeInfo) (coreclient.Client, error)
}
}
type reputationClient struct {
coreclient.MultiAddressClient
prm truststorage.UpdatePrm
cons *reputationClientConstructor
}
func (c *reputationClient) submitResult(err error) {
currEpoch := c.cons.netState.CurrentEpoch()
sat := err == nil
c.cons.log.Debug(
"writing local reputation values",
zap.Uint64("epoch", currEpoch),
zap.Bool("satisfactory", sat),
)
prm := c.prm
prm.SetSatisfactory(sat)
prm.SetEpoch(currEpoch)
c.cons.trustStorage.Update(prm)
}
func (c *reputationClient) ObjectPutInit(ctx context.Context, prm client.PrmObjectPutInit) (*client.ObjectWriter, error) {
res, err := c.MultiAddressClient.ObjectPutInit(ctx, prm)
// FIXME: (neofs-node#1193) here we submit only initialization errors, writing errors are not processed
c.submitResult(err)
return res, err
}
func (c *reputationClient) ObjectDelete(ctx context.Context, prm client.PrmObjectDelete) (*client.ResObjectDelete, error) {
res, err := c.MultiAddressClient.ObjectDelete(ctx, prm)
if err != nil {
c.submitResult(err)
} else {
c.submitResult(apistatus.ErrFromStatus(res.Status()))
}
return res, err
}
func (c *reputationClient) GetObjectInit(ctx context.Context, prm client.PrmObjectGet) (*client.ObjectReader, error) {
res, err := c.MultiAddressClient.ObjectGetInit(ctx, prm)
// FIXME: (neofs-node#1193) here we submit only initialization errors, reading errors are not processed
c.submitResult(err)
return res, err
}
func (c *reputationClient) ObjectHead(ctx context.Context, prm client.PrmObjectHead) (*client.ResObjectHead, error) {
res, err := c.MultiAddressClient.ObjectHead(ctx, prm)
c.submitResult(err)
return res, err
}
func (c *reputationClient) ObjectHash(ctx context.Context, prm client.PrmObjectHash) (*client.ResObjectHash, error) {
res, err := c.MultiAddressClient.ObjectHash(ctx, prm)
c.submitResult(err)
return res, err
}
func (c *reputationClient) ObjectSearchInit(ctx context.Context, prm client.PrmObjectSearch) (*client.ObjectListReader, error) {
res, err := c.MultiAddressClient.ObjectSearchInit(ctx, prm)
// FIXME: (neofs-node#1193) here we submit only initialization errors, reading errors are not processed
c.submitResult(err)
return res, err
}
func (c *reputationClientConstructor) Get(info coreclient.NodeInfo) (coreclient.Client, error) {
cl, err := c.basicConstructor.Get(info)
if err != nil {
return nil, err
}
nm, err := netmap.GetLatestNetworkMap(c.nmSrc)
if err == nil {
key := info.PublicKey()
nmNodes := nm.Nodes()
var peer apireputation.PeerID
for i := range nmNodes {
if bytes.Equal(nmNodes[i].PublicKey(), key) {
peer.SetPublicKey(nmNodes[i].PublicKey())
prm := truststorage.UpdatePrm{}
prm.SetPeer(peer)
return &reputationClient{
MultiAddressClient: cl.(coreclient.MultiAddressClient),
prm: prm,
cons: c,
}, nil
}
}
} else {
c.log.Warn("could not get latest network map to overload the client",
zap.String("error", err.Error()),
)
}
return cl, nil
}
type engineWithNotifications struct {
base putsvc.ObjectStorage
nw notificationWriter
ns netmap.State
defaultTopic string
}
func (e engineWithNotifications) Delete(tombstone oid.Address, toDelete []oid.ID) error {
return e.base.Delete(tombstone, toDelete)
}
func (e engineWithNotifications) Lock(locker oid.Address, toLock []oid.ID) error {
return e.base.Lock(locker, toLock)
}
func (e engineWithNotifications) Put(o *objectSDK.Object) error {
if err := e.base.Put(o); err != nil {
return err
}
ni, err := o.NotificationInfo()
if err == nil {
if epoch := ni.Epoch(); epoch == 0 || epoch == e.ns.CurrentEpoch() {
topic := ni.Topic()
if topic == "" {
topic = e.defaultTopic
}
e.nw.Notify(topic, objectCore.AddressOf(o))
}
}
return nil
}
type engineWithoutNotifications struct {
engine *engine.StorageEngine
}
func (e engineWithoutNotifications) Delete(tombstone oid.Address, toDelete []oid.ID) error {
var prm engine.InhumePrm
addrs := make([]oid.Address, len(toDelete))
for i := range addrs {
addrs[i].SetContainer(tombstone.Container())
addrs[i].SetObject(toDelete[i])
}
prm.WithTarget(tombstone, addrs...)
_, err := e.engine.Inhume(prm)
return err
}
func (e engineWithoutNotifications) Lock(locker oid.Address, toLock []oid.ID) error {
return e.engine.Lock(locker.Container(), locker.Object(), toLock)
}
func (e engineWithoutNotifications) Put(o *objectSDK.Object) error {
return engine.Put(e.engine, o)
}

46
cmd/frostfs-node/pprof.go Normal file
View file

@ -0,0 +1,46 @@
package main
import (
"context"
profilerconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/profiler"
httputil "github.com/TrueCloudLab/frostfs-node/pkg/util/http"
"go.uber.org/zap"
)
func initProfiler(c *cfg) {
if !profilerconfig.Enabled(c.appCfg) {
c.log.Info("pprof is disabled")
return
}
var prm httputil.Prm
prm.Address = profilerconfig.Address(c.appCfg)
prm.Handler = httputil.Handler()
srv := httputil.New(prm,
httputil.WithShutdownTimeout(
profilerconfig.ShutdownTimeout(c.appCfg),
),
)
c.workers = append(c.workers, newWorkerFromFunc(func(context.Context) {
runAndLog(c, "profiler", false, func(c *cfg) {
fatalOnErr(srv.Serve())
})
}))
c.closers = append(c.closers, func() {
c.log.Debug("shutting down profiling service")
err := srv.Shutdown()
if err != nil {
c.log.Debug("could not shutdown pprof server",
zap.String("error", err.Error()),
)
}
c.log.Debug("profiling service has been stopped")
})
}

View file

@ -0,0 +1,357 @@
package main
import (
"context"
"fmt"
v2reputation "github.com/TrueCloudLab/frostfs-api-go/v2/reputation"
v2reputationgrpc "github.com/TrueCloudLab/frostfs-api-go/v2/reputation/grpc"
"github.com/TrueCloudLab/frostfs-api-go/v2/session"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/reputation/common"
intermediatereputation "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/reputation/intermediate"
localreputation "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/reputation/local"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/reputation/ticker"
repClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client/reputation"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/event"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/event/netmap"
grpcreputation "github.com/TrueCloudLab/frostfs-node/pkg/network/transport/reputation/grpc"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation"
reputationcommon "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/common"
reputationrouter "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/common/router"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust"
eigentrustcalc "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/calculator"
eigentrustctrl "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/controller"
intermediateroutes "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/routes"
consumerstorage "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/storage/consumers"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/storage/daughters"
localtrustcontroller "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/local/controller"
localroutes "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/local/routes"
truststorage "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/local/storage"
reputationrpc "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/rpc"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
apireputation "github.com/TrueCloudLab/frostfs-sdk-go/reputation"
"go.uber.org/zap"
)
func initReputationService(c *cfg) {
wrap, err := repClient.NewFromMorph(c.cfgMorph.client, c.cfgReputation.scriptHash, 0, repClient.TryNotary())
fatalOnErr(err)
localKey := c.key.PublicKey().Bytes()
nmSrc := c.netMapSource
// storing calculated trusts as a daughter
c.cfgReputation.localTrustStorage = truststorage.New(
truststorage.Prm{},
)
daughterStorage := daughters.New(daughters.Prm{})
consumerStorage := consumerstorage.New(consumerstorage.Prm{})
// storing received daughter(of current node) trusts as a manager
daughterStorageWriterProvider := &intermediatereputation.DaughterStorageWriterProvider{
Log: c.log,
Storage: daughterStorage,
}
consumerStorageWriterProvider := &intermediatereputation.ConsumerStorageWriterProvider{
Log: c.log,
Storage: consumerStorage,
}
localTrustLogger := &logger.Logger{Logger: c.log.With(zap.String("trust_type", "local"))}
intermediateTrustLogger := &logger.Logger{Logger: c.log.With(zap.String("trust_type", "intermediate"))}
localTrustStorage := &localreputation.TrustStorage{
Log: localTrustLogger,
Storage: c.cfgReputation.localTrustStorage,
NmSrc: nmSrc,
LocalKey: localKey,
}
managerBuilder := reputationcommon.NewManagerBuilder(
reputationcommon.ManagersPrm{
NetMapSource: nmSrc,
},
reputationcommon.WithLogger(c.log),
)
localRouteBuilder := localroutes.New(
localroutes.Prm{
ManagerBuilder: managerBuilder,
Log: localTrustLogger,
},
)
intermediateRouteBuilder := intermediateroutes.New(
intermediateroutes.Prm{
ManagerBuilder: managerBuilder,
Log: intermediateTrustLogger,
},
)
remoteLocalTrustProvider := common.NewRemoteTrustProvider(
common.RemoteProviderPrm{
NetmapKeys: c,
DeadEndProvider: daughterStorageWriterProvider,
ClientCache: c.bgClientCache,
WriterProvider: localreputation.NewRemoteProvider(
localreputation.RemoteProviderPrm{
Key: &c.key.PrivateKey,
Log: localTrustLogger,
},
),
Log: localTrustLogger,
},
)
remoteIntermediateTrustProvider := common.NewRemoteTrustProvider(
common.RemoteProviderPrm{
NetmapKeys: c,
DeadEndProvider: consumerStorageWriterProvider,
ClientCache: c.bgClientCache,
WriterProvider: intermediatereputation.NewRemoteProvider(
intermediatereputation.RemoteProviderPrm{
Key: &c.key.PrivateKey,
Log: intermediateTrustLogger,
},
),
Log: intermediateTrustLogger,
},
)
localTrustRouter := reputationrouter.New(
reputationrouter.Prm{
LocalServerInfo: c,
RemoteWriterProvider: remoteLocalTrustProvider,
Builder: localRouteBuilder,
},
reputationrouter.WithLogger(localTrustLogger))
intermediateTrustRouter := reputationrouter.New(
reputationrouter.Prm{
LocalServerInfo: c,
RemoteWriterProvider: remoteIntermediateTrustProvider,
Builder: intermediateRouteBuilder,
},
reputationrouter.WithLogger(intermediateTrustLogger),
)
eigenTrustCalculator := eigentrustcalc.New(
eigentrustcalc.Prm{
AlphaProvider: c.cfgNetmap.wrapper,
InitialTrustSource: intermediatereputation.InitialTrustSource{
NetMap: nmSrc,
},
IntermediateValueTarget: intermediateTrustRouter,
WorkerPool: c.cfgReputation.workerPool,
FinalResultTarget: intermediatereputation.NewFinalWriterProvider(
intermediatereputation.FinalWriterProviderPrm{
PrivatKey: &c.key.PrivateKey,
PubKey: localKey,
Client: wrap,
},
intermediatereputation.FinalWriterWithLogger(c.log),
),
DaughterTrustSource: &intermediatereputation.DaughterTrustIteratorProvider{
DaughterStorage: daughterStorage,
ConsumerStorage: consumerStorage,
},
},
eigentrustcalc.WithLogger(c.log),
)
eigenTrustController := eigentrustctrl.New(
eigentrustctrl.Prm{
DaughtersTrustCalculator: &intermediatereputation.DaughtersTrustCalculator{
Calculator: eigenTrustCalculator,
},
IterationsProvider: c.cfgNetmap.wrapper,
WorkerPool: c.cfgReputation.workerPool,
},
eigentrustctrl.WithLogger(c.log),
)
c.cfgReputation.localTrustCtrl = localtrustcontroller.New(
localtrustcontroller.Prm{
LocalTrustSource: localTrustStorage,
LocalTrustTarget: localTrustRouter,
},
localtrustcontroller.WithLogger(c.log),
)
addNewEpochAsyncNotificationHandler(
c,
func(ev event.Event) {
c.log.Debug("start reporting reputation on new epoch event")
var reportPrm localtrustcontroller.ReportPrm
// report collected values from previous epoch
reportPrm.SetEpoch(ev.(netmap.NewEpoch).EpochNumber() - 1)
c.cfgReputation.localTrustCtrl.Report(reportPrm)
},
)
server := grpcreputation.New(
reputationrpc.NewSignService(
&c.key.PrivateKey,
reputationrpc.NewResponseService(
&reputationServer{
cfg: c,
log: c.log,
localRouter: localTrustRouter,
intermediateRouter: intermediateTrustRouter,
routeBuilder: localRouteBuilder,
},
c.respSvc,
),
),
)
for _, srv := range c.cfgGRPC.servers {
v2reputationgrpc.RegisterReputationServiceServer(srv, server)
}
// initialize eigen trust block timer
newEigenTrustIterTimer(c)
addNewEpochAsyncNotificationHandler(
c,
func(e event.Event) {
epoch := e.(netmap.NewEpoch).EpochNumber()
log := c.log.With(zap.Uint64("epoch", epoch))
duration, err := c.cfgNetmap.wrapper.EpochDuration()
if err != nil {
log.Debug("could not fetch epoch duration", zap.Error(err))
return
}
iterations, err := c.cfgNetmap.wrapper.EigenTrustIterations()
if err != nil {
log.Debug("could not fetch iteration number", zap.Error(err))
return
}
epochTimer, err := ticker.NewIterationsTicker(duration, iterations, func() {
eigenTrustController.Continue(
eigentrustctrl.ContinuePrm{
Epoch: epoch - 1,
},
)
})
if err != nil {
log.Debug("could not create fixed epoch timer", zap.Error(err))
return
}
c.cfgMorph.eigenTrustTicker.addEpochTimer(epoch, epochTimer)
},
)
}
type reputationServer struct {
*cfg
log *logger.Logger
localRouter reputationcommon.WriterProvider
intermediateRouter reputationcommon.WriterProvider
routeBuilder reputationrouter.Builder
}
func (s *reputationServer) AnnounceLocalTrust(ctx context.Context, req *v2reputation.AnnounceLocalTrustRequest) (*v2reputation.AnnounceLocalTrustResponse, error) {
passedRoute := reverseRoute(req.GetVerificationHeader())
passedRoute = append(passedRoute, s)
body := req.GetBody()
eCtx := &common.EpochContext{
Context: ctx,
E: body.GetEpoch(),
}
w, err := s.localRouter.InitWriter(reputationrouter.NewRouteContext(eCtx, passedRoute))
if err != nil {
return nil, fmt.Errorf("could not initialize local trust writer: %w", err)
}
for _, trust := range body.GetTrusts() {
err = s.processLocalTrust(body.GetEpoch(), apiToLocalTrust(&trust, passedRoute[0].PublicKey()), passedRoute, w)
if err != nil {
return nil, fmt.Errorf("could not write one of local trusts: %w", err)
}
}
resp := new(v2reputation.AnnounceLocalTrustResponse)
resp.SetBody(new(v2reputation.AnnounceLocalTrustResponseBody))
return resp, nil
}
func (s *reputationServer) AnnounceIntermediateResult(ctx context.Context, req *v2reputation.AnnounceIntermediateResultRequest) (*v2reputation.AnnounceIntermediateResultResponse, error) {
passedRoute := reverseRoute(req.GetVerificationHeader())
passedRoute = append(passedRoute, s)
body := req.GetBody()
eiCtx := eigentrust.NewIterContext(ctx, body.GetEpoch(), body.GetIteration())
w, err := s.intermediateRouter.InitWriter(reputationrouter.NewRouteContext(eiCtx, passedRoute))
if err != nil {
return nil, fmt.Errorf("could not initialize trust writer: %w", err)
}
v2Trust := body.GetTrust()
trust := apiToLocalTrust(v2Trust.GetTrust(), v2Trust.GetTrustingPeer().GetPublicKey())
err = w.Write(trust)
if err != nil {
return nil, fmt.Errorf("could not write trust: %w", err)
}
resp := new(v2reputation.AnnounceIntermediateResultResponse)
resp.SetBody(new(v2reputation.AnnounceIntermediateResultResponseBody))
return resp, nil
}
func (s *reputationServer) processLocalTrust(epoch uint64, t reputation.Trust,
passedRoute []reputationcommon.ServerInfo, w reputationcommon.Writer) error {
err := reputationrouter.CheckRoute(s.routeBuilder, epoch, t, passedRoute)
if err != nil {
return fmt.Errorf("wrong route of reputation trust value: %w", err)
}
return w.Write(t)
}
// apiToLocalTrust converts v2 Trust to local reputation.Trust, adding trustingPeer.
func apiToLocalTrust(t *v2reputation.Trust, trustingPeer []byte) reputation.Trust {
var trusted, trusting apireputation.PeerID
trusted.SetPublicKey(t.GetPeer().GetPublicKey())
trusting.SetPublicKey(trustingPeer)
localTrust := reputation.Trust{}
localTrust.SetValue(reputation.TrustValueFromFloat64(t.GetValue()))
localTrust.SetPeer(trusted)
localTrust.SetTrustingPeer(trusting)
return localTrust
}
func reverseRoute(hdr *session.RequestVerificationHeader) (passedRoute []reputationcommon.ServerInfo) {
for hdr != nil {
passedRoute = append(passedRoute, &common.OnlyKeyRemoteServerInfo{
Key: hdr.GetBodySignature().GetKey(),
})
hdr = hdr.GetOrigin()
}
return
}

View file

@ -0,0 +1,101 @@
package common
import (
"fmt"
"github.com/TrueCloudLab/frostfs-node/pkg/core/client"
"github.com/TrueCloudLab/frostfs-node/pkg/core/netmap"
reputationcommon "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/common"
reputationrouter "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/common/router"
trustcontroller "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/local/controller"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
)
type clientCache interface {
Get(client.NodeInfo) (client.Client, error)
}
// clientKeyRemoteProvider must provide a remote writer and take into account
// that requests must be sent via the passed api client and must be signed with
// the passed private key.
type clientKeyRemoteProvider interface {
WithClient(client.Client) reputationcommon.WriterProvider
}
// remoteTrustProvider is an implementation of reputation RemoteWriterProvider interface.
// It caches clients, checks if it is the end of the route and checks either the current
// node is a remote target or not.
//
// remoteTrustProvider requires to be provided with clientKeyRemoteProvider.
type remoteTrustProvider struct {
netmapKeys netmap.AnnouncedKeys
deadEndProvider reputationcommon.WriterProvider
clientCache clientCache
remoteProvider clientKeyRemoteProvider
log *logger.Logger
}
// RemoteProviderPrm groups the required parameters of the remoteTrustProvider's constructor.
//
// All values must comply with the requirements imposed on them.
// Passing incorrect parameter values will result in constructor
// failure (error or panic depending on the implementation).
type RemoteProviderPrm struct {
NetmapKeys netmap.AnnouncedKeys
DeadEndProvider reputationcommon.WriterProvider
ClientCache clientCache
WriterProvider clientKeyRemoteProvider
Log *logger.Logger
}
func NewRemoteTrustProvider(prm RemoteProviderPrm) reputationrouter.RemoteWriterProvider {
switch {
case prm.NetmapKeys == nil:
PanicOnPrmValue("NetmapKeys", prm.NetmapKeys)
case prm.DeadEndProvider == nil:
PanicOnPrmValue("DeadEndProvider", prm.DeadEndProvider)
case prm.ClientCache == nil:
PanicOnPrmValue("ClientCache", prm.ClientCache)
case prm.WriterProvider == nil:
PanicOnPrmValue("WriterProvider", prm.WriterProvider)
case prm.Log == nil:
PanicOnPrmValue("Logger", prm.Log)
}
return &remoteTrustProvider{
netmapKeys: prm.NetmapKeys,
deadEndProvider: prm.DeadEndProvider,
clientCache: prm.ClientCache,
remoteProvider: prm.WriterProvider,
log: prm.Log,
}
}
func (rtp *remoteTrustProvider) InitRemote(srv reputationcommon.ServerInfo) (reputationcommon.WriterProvider, error) {
rtp.log.Debug("initializing remote writer provider")
if srv == nil {
rtp.log.Debug("route has reached dead-end provider")
return rtp.deadEndProvider, nil
}
if rtp.netmapKeys.IsLocalKey(srv.PublicKey()) {
// if local => return no-op writer
rtp.log.Debug("initializing no-op writer provider")
return trustcontroller.SimpleWriterProvider(new(NopReputationWriter)), nil
}
var info client.NodeInfo
err := client.NodeInfoFromRawNetmapElement(&info, srv)
if err != nil {
return nil, fmt.Errorf("parse client node info: %w", err)
}
c, err := rtp.clientCache.Get(info)
if err != nil {
return nil, fmt.Errorf("could not initialize API client: %w", err)
}
return rtp.remoteProvider.WithClient(c), nil
}

View file

@ -0,0 +1,55 @@
package common
import (
"context"
"fmt"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation"
)
// EpochContext is a std context extended with epoch data.
type EpochContext struct {
context.Context
E uint64
}
func (ctx *EpochContext) Epoch() uint64 {
return ctx.E
}
type NopReputationWriter struct{}
func (NopReputationWriter) Write(reputation.Trust) error {
return nil
}
func (NopReputationWriter) Close() error {
return nil
}
// OnlyKeyRemoteServerInfo is an implementation of reputation.ServerInfo
// interface but with only public key data.
type OnlyKeyRemoteServerInfo struct {
Key []byte
}
func (i *OnlyKeyRemoteServerInfo) PublicKey() []byte {
return i.Key
}
func (*OnlyKeyRemoteServerInfo) IterateAddresses(func(string) bool) {
}
func (*OnlyKeyRemoteServerInfo) NumberOfAddresses() int {
return 0
}
func (*OnlyKeyRemoteServerInfo) ExternalAddresses() []string {
return nil
}
const invalidPrmValFmt = "invalid parameter %s (%T):%v"
func PanicOnPrmValue(n string, v interface{}) {
panic(fmt.Sprintf(invalidPrmValFmt, n, v, v))
}

View file

@ -0,0 +1,56 @@
package intermediate
import (
"errors"
"fmt"
"github.com/TrueCloudLab/frostfs-node/pkg/core/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust"
eigencalc "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/calculator"
eigentrustctrl "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/controller"
apireputation "github.com/TrueCloudLab/frostfs-sdk-go/reputation"
)
// InitialTrustSource is an implementation of the
// reputation/eigentrust/calculator's InitialTrustSource interface.
type InitialTrustSource struct {
NetMap netmap.Source
}
var ErrEmptyNetMap = errors.New("empty NepMap")
// InitialTrust returns `initialTrust` as an initial trust value.
func (i InitialTrustSource) InitialTrust(apireputation.PeerID) (reputation.TrustValue, error) {
nm, err := i.NetMap.GetNetMap(1)
if err != nil {
return reputation.TrustZero, fmt.Errorf("failed to get NetMap: %w", err)
}
nodeCount := reputation.TrustValueFromFloat64(float64(len(nm.Nodes())))
if nodeCount == 0 {
return reputation.TrustZero, ErrEmptyNetMap
}
return reputation.TrustOne.Div(nodeCount), nil
}
// DaughtersTrustCalculator wraps EigenTrust calculator and implements the
// eigentrust/calculator's DaughtersTrustCalculator interface.
type DaughtersTrustCalculator struct {
Calculator *eigencalc.Calculator
}
// Calculate converts and passes values to the wrapped calculator.
func (c *DaughtersTrustCalculator) Calculate(ctx eigentrustctrl.IterationContext) {
calcPrm := eigencalc.CalculatePrm{}
epochIteration := eigentrust.EpochIteration{}
epochIteration.SetEpoch(ctx.Epoch())
epochIteration.SetI(ctx.I())
calcPrm.SetLast(ctx.Last())
calcPrm.SetEpochIteration(epochIteration)
c.Calculator.Calculate(calcPrm)
}

View file

@ -0,0 +1,63 @@
package intermediate
import (
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation"
reputationcommon "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/common"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust"
eigencalc "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/calculator"
consumerstorage "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/storage/consumers"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
"go.uber.org/zap"
)
var ErrIncorrectContextPanicMsg = "could not write intermediate trust: passed context incorrect"
// ConsumerStorageWriterProvider is an implementation of the reputation.WriterProvider
// interface that provides ConsumerTrustWriter writer.
type ConsumerStorageWriterProvider struct {
Log *logger.Logger
Storage *consumerstorage.Storage
}
// ConsumerTrustWriter is an implementation of the reputation.Writer interface
// that writes passed consumer's Trust values to the Consumer storage. After writing
// that, values can be used in eigenTrust algorithm's iterations.
type ConsumerTrustWriter struct {
log *logger.Logger
storage *consumerstorage.Storage
eiCtx eigencalc.Context
}
func (w *ConsumerTrustWriter) Write(t reputation.Trust) error {
w.log.Debug("writing received consumer's trusts",
zap.Uint64("epoch", w.eiCtx.Epoch()),
zap.Uint32("iteration", w.eiCtx.I()),
zap.Stringer("trusting_peer", t.TrustingPeer()),
zap.Stringer("trusted_peer", t.Peer()),
)
trust := eigentrust.IterationTrust{Trust: t}
trust.SetEpoch(w.eiCtx.Epoch())
trust.SetI(w.eiCtx.I())
w.storage.Put(trust)
return nil
}
func (w *ConsumerTrustWriter) Close() error {
return nil
}
func (s *ConsumerStorageWriterProvider) InitWriter(ctx reputationcommon.Context) (reputationcommon.Writer, error) {
eiCtx, ok := ctx.(eigencalc.Context)
if !ok {
panic(ErrIncorrectContextPanicMsg)
}
return &ConsumerTrustWriter{
log: s.Log,
storage: s.Storage,
eiCtx: eiCtx,
}, nil
}

View file

@ -0,0 +1,146 @@
package intermediate
import (
"crypto/ecdsa"
"fmt"
repClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client/reputation"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust"
eigentrustcalc "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/calculator"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
frostfsecdsa "github.com/TrueCloudLab/frostfs-sdk-go/crypto/ecdsa"
apireputation "github.com/TrueCloudLab/frostfs-sdk-go/reputation"
"go.uber.org/zap"
)
// FinalWriterProviderPrm groups the required parameters of the FinalWriterProvider's constructor.
//
// All values must comply with the requirements imposed on them.
// Passing incorrect parameter values will result in constructor
// failure (error or panic depending on the implementation).
type FinalWriterProviderPrm struct {
PrivatKey *ecdsa.PrivateKey
PubKey []byte
Client *repClient.Client
}
// NewFinalWriterProvider creates a new instance of the FinalWriterProvider.
//
// Panics if at least one value of the parameters is invalid.
//
// The created FinalWriterProvider does not require additional
// initialization and is completely ready for work.
func NewFinalWriterProvider(prm FinalWriterProviderPrm, opts ...FinalWriterOption) *FinalWriterProvider {
o := defaultFinalWriterOptionsOpts()
for i := range opts {
opts[i](o)
}
return &FinalWriterProvider{
prm: prm,
opts: o,
}
}
// FinalWriterProvider is an implementation of the reputation.eigentrust.calculator
// IntermediateWriterProvider interface. It inits FinalWriter.
type FinalWriterProvider struct {
prm FinalWriterProviderPrm
opts *finalWriterOptions
}
func (fwp FinalWriterProvider) InitIntermediateWriter(
_ eigentrustcalc.Context) (eigentrustcalc.IntermediateWriter, error) {
return &FinalWriter{
privatKey: fwp.prm.PrivatKey,
pubKey: fwp.prm.PubKey,
client: fwp.prm.Client,
l: fwp.opts.log,
}, nil
}
// FinalWriter is an implementation of the reputation.eigentrust.calculator IntermediateWriter
// interface that writes GlobalTrust to contract directly.
type FinalWriter struct {
privatKey *ecdsa.PrivateKey
pubKey []byte
client *repClient.Client
l *logger.Logger
}
func (fw FinalWriter) WriteIntermediateTrust(t eigentrust.IterationTrust) error {
fw.l.Debug("start writing global trusts to contract")
args := repClient.PutPrm{}
apiTrustedPeerID := t.Peer()
var apiTrust apireputation.Trust
apiTrust.SetValue(t.Value().Float64())
apiTrust.SetPeer(t.Peer())
var managerPublicKey [33]byte
copy(managerPublicKey[:], fw.pubKey)
var apiMangerPeerID apireputation.PeerID
apiMangerPeerID.SetPublicKey(managerPublicKey[:])
var gTrust apireputation.GlobalTrust
gTrust.SetTrust(apiTrust)
gTrust.SetManager(apiMangerPeerID)
err := gTrust.Sign(frostfsecdsa.Signer(*fw.privatKey))
if err != nil {
fw.l.Debug(
"failed to sign global trust",
zap.Error(err),
)
return fmt.Errorf("failed to sign global trust: %w", err)
}
args.SetEpoch(t.Epoch())
args.SetValue(gTrust)
args.SetPeerID(apiTrustedPeerID)
err = fw.client.Put(
args,
)
if err != nil {
fw.l.Debug(
"failed to write global trust to contract",
zap.Error(err),
)
return fmt.Errorf("failed to write global trust to contract: %w", err)
}
fw.l.Debug(
"sent global trust to contract",
zap.Uint64("epoch", t.Epoch()),
zap.Float64("value", t.Value().Float64()),
zap.Stringer("peer", t.Peer()),
)
return nil
}
type finalWriterOptions struct {
log *logger.Logger
}
type FinalWriterOption func(*finalWriterOptions)
func defaultFinalWriterOptionsOpts() *finalWriterOptions {
return &finalWriterOptions{
log: &logger.Logger{Logger: zap.L()},
}
}
func FinalWriterWithLogger(l *logger.Logger) FinalWriterOption {
return func(o *finalWriterOptions) {
if l != nil {
o.log = l
}
}
}

View file

@ -0,0 +1,48 @@
package intermediate
import (
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation"
reputationcommon "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/common"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/storage/daughters"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
"go.uber.org/zap"
)
// DaughterStorageWriterProvider is an implementation of the reputation.WriterProvider
// interface that provides DaughterTrustWriter writer.
type DaughterStorageWriterProvider struct {
Log *logger.Logger
Storage *daughters.Storage
}
// DaughterTrustWriter is an implementation of the reputation.Writer interface
// that writes passed daughter's Trust values to Daughter storage. After writing
// that, values can be used in eigenTrust algorithm's iterations.
type DaughterTrustWriter struct {
log *logger.Logger
storage *daughters.Storage
ctx reputationcommon.Context
}
func (w *DaughterTrustWriter) Write(t reputation.Trust) error {
w.log.Debug("writing received daughter's trusts",
zap.Uint64("epoch", w.ctx.Epoch()),
zap.Stringer("trusting_peer", t.TrustingPeer()),
zap.Stringer("trusted_peer", t.Peer()),
)
w.storage.Put(w.ctx.Epoch(), t)
return nil
}
func (w *DaughterTrustWriter) Close() error {
return nil
}
func (s *DaughterStorageWriterProvider) InitWriter(ctx reputationcommon.Context) (reputationcommon.Writer, error) {
return &DaughterTrustWriter{
log: s.Log,
storage: s.Storage,
ctx: ctx,
}, nil
}

View file

@ -0,0 +1,124 @@
package intermediate
import (
"crypto/ecdsa"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/reputation/common"
internalclient "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/reputation/internal/client"
coreclient "github.com/TrueCloudLab/frostfs-node/pkg/core/client"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation"
reputationcommon "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/common"
eigentrustcalc "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/calculator"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
reputationapi "github.com/TrueCloudLab/frostfs-sdk-go/reputation"
"go.uber.org/zap"
)
// RemoteProviderPrm groups the required parameters of the RemoteProvider's constructor.
//
// All values must comply with the requirements imposed on them.
// Passing incorrect parameter values will result in constructor
// failure (error or panic depending on the implementation).
type RemoteProviderPrm struct {
Key *ecdsa.PrivateKey
Log *logger.Logger
}
// NewRemoteProvider creates a new instance of the RemoteProvider.
//
// Panics if at least one value of the parameters is invalid.
//
// The created RemoteProvider does not require additional
// initialization and is completely ready for work.
func NewRemoteProvider(prm RemoteProviderPrm) *RemoteProvider {
switch {
case prm.Key == nil:
common.PanicOnPrmValue("NetMapSource", prm.Key)
case prm.Log == nil:
common.PanicOnPrmValue("Logger", prm.Log)
}
return &RemoteProvider{
key: prm.Key,
log: prm.Log,
}
}
// RemoteProvider is an implementation of the clientKeyRemoteProvider interface.
type RemoteProvider struct {
key *ecdsa.PrivateKey
log *logger.Logger
}
func (rp RemoteProvider) WithClient(c coreclient.Client) reputationcommon.WriterProvider {
return &TrustWriterProvider{
client: c,
key: rp.key,
log: rp.log,
}
}
type TrustWriterProvider struct {
client coreclient.Client
key *ecdsa.PrivateKey
log *logger.Logger
}
func (twp *TrustWriterProvider) InitWriter(ctx reputationcommon.Context) (reputationcommon.Writer, error) {
eiContext, ok := ctx.(eigentrustcalc.Context)
if !ok {
// TODO: #1164 think if this can be done without such limitation
panic(ErrIncorrectContextPanicMsg)
}
return &RemoteTrustWriter{
eiCtx: eiContext,
client: twp.client,
key: twp.key,
log: twp.log,
}, nil
}
type RemoteTrustWriter struct {
eiCtx eigentrustcalc.Context
client coreclient.Client
key *ecdsa.PrivateKey
log *logger.Logger
}
// Write sends a trust value to a remote node via ReputationService.AnnounceIntermediateResult RPC.
func (rtp *RemoteTrustWriter) Write(t reputation.Trust) error {
epoch := rtp.eiCtx.Epoch()
i := rtp.eiCtx.I()
rtp.log.Debug("announcing trust",
zap.Uint64("epoch", epoch),
zap.Uint32("iteration", i),
zap.Stringer("trusting_peer", t.TrustingPeer()),
zap.Stringer("trusted_peer", t.Peer()),
)
var apiTrust reputationapi.Trust
apiTrust.SetValue(t.Value().Float64())
apiTrust.SetPeer(t.Peer())
var apiPeerToPeerTrust reputationapi.PeerToPeerTrust
apiPeerToPeerTrust.SetTrustingPeer(t.TrustingPeer())
apiPeerToPeerTrust.SetTrust(apiTrust)
var p internalclient.AnnounceIntermediatePrm
p.SetContext(rtp.eiCtx)
p.SetClient(rtp.client)
p.SetEpoch(epoch)
p.SetIteration(i)
p.SetTrust(apiPeerToPeerTrust)
_, err := internalclient.AnnounceIntermediate(p)
return err
}
func (rtp *RemoteTrustWriter) Close() error {
return nil
}

View file

@ -0,0 +1,64 @@
package intermediate
import (
"fmt"
eigentrustcalc "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/calculator"
consumerstorage "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/storage/consumers"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/eigentrust/storage/daughters"
apireputation "github.com/TrueCloudLab/frostfs-sdk-go/reputation"
)
// DaughterTrustIteratorProvider is an implementation of the
// reputation/eigentrust/calculator's DaughterTrustIteratorProvider interface.
type DaughterTrustIteratorProvider struct {
DaughterStorage *daughters.Storage
ConsumerStorage *consumerstorage.Storage
}
// InitDaughterIterator returns an iterator over the received
// local trusts for ctx.Epoch() epoch from daughter p.
func (ip *DaughterTrustIteratorProvider) InitDaughterIterator(ctx eigentrustcalc.Context,
p apireputation.PeerID) (eigentrustcalc.TrustIterator, error) {
epoch := ctx.Epoch()
daughterIterator, ok := ip.DaughterStorage.DaughterTrusts(epoch, p)
if !ok {
return nil, fmt.Errorf("no data in %d epoch for daughter: %s", epoch, p)
}
return daughterIterator, nil
}
// InitAllDaughtersIterator returns an iterator over all
// daughters of the current node(manager) and all local
// trusts received from them for ctx.Epoch() epoch.
func (ip *DaughterTrustIteratorProvider) InitAllDaughtersIterator(
ctx eigentrustcalc.Context) (eigentrustcalc.PeerTrustsIterator, error) {
epoch := ctx.Epoch()
iter, ok := ip.DaughterStorage.AllDaughterTrusts(epoch)
if !ok {
return nil, fmt.Errorf("no data in %d epoch for daughters", epoch)
}
return iter, nil
}
// InitConsumersIterator returns an iterator over all daughters
// of the current node(manager) and all their consumers' local
// trusts for ctx.Epoch() epoch and ctx.I() iteration.
func (ip *DaughterTrustIteratorProvider) InitConsumersIterator(
ctx eigentrustcalc.Context) (eigentrustcalc.PeerTrustsIterator, error) {
epoch, iter := ctx.Epoch(), ctx.I()
consumerIterator, ok := ip.ConsumerStorage.Consumers(epoch, iter)
if !ok {
return nil, fmt.Errorf("no data for %d iteration in %d epoch for consumers's trusts",
iter,
epoch,
)
}
return consumerIterator, nil
}

View file

@ -0,0 +1,110 @@
package internal
import (
"context"
coreclient "github.com/TrueCloudLab/frostfs-node/pkg/core/client"
"github.com/TrueCloudLab/frostfs-sdk-go/client"
apistatus "github.com/TrueCloudLab/frostfs-sdk-go/client/status"
"github.com/TrueCloudLab/frostfs-sdk-go/reputation"
)
type commonPrm struct {
cli coreclient.Client
ctx context.Context
}
// SetClient sets the base client for NeoFS API communication.
//
// Required parameter.
func (x *commonPrm) SetClient(cli coreclient.Client) {
x.cli = cli
}
// SetContext sets context.Context for network communication.
//
// Required parameter.
func (x *commonPrm) SetContext(ctx context.Context) {
x.ctx = ctx
}
// AnnounceLocalPrm groups parameters of AnnounceLocal operation.
type AnnounceLocalPrm struct {
commonPrm
cliPrm client.PrmAnnounceLocalTrust
}
// SetEpoch sets the epoch in which the trust was assessed.
func (x *AnnounceLocalPrm) SetEpoch(epoch uint64) {
x.cliPrm.SetEpoch(epoch)
}
// SetTrusts sets a list of local trust values.
func (x *AnnounceLocalPrm) SetTrusts(ts []reputation.Trust) {
x.cliPrm.SetValues(ts)
}
// AnnounceLocalRes groups the resulting values of AnnounceLocal operation.
type AnnounceLocalRes struct{}
// AnnounceLocal sends estimations of local trust to the remote node.
//
// Client, context and key must be set.
//
// Returns any error which prevented the operation from completing correctly in error return.
func AnnounceLocal(prm AnnounceLocalPrm) (res AnnounceLocalRes, err error) {
var cliRes *client.ResAnnounceLocalTrust
cliRes, err = prm.cli.AnnounceLocalTrust(prm.ctx, prm.cliPrm)
if err == nil {
// pull out an error from status
err = apistatus.ErrFromStatus(cliRes.Status())
}
return
}
// AnnounceIntermediatePrm groups parameters of AnnounceIntermediate operation.
type AnnounceIntermediatePrm struct {
commonPrm
cliPrm client.PrmAnnounceIntermediateTrust
}
// SetEpoch sets the number of the epoch when the trust calculation's iteration was executed.
func (x *AnnounceIntermediatePrm) SetEpoch(epoch uint64) {
x.cliPrm.SetEpoch(epoch)
}
// SetIteration sets the number of the iteration of the trust calculation algorithm.
func (x *AnnounceIntermediatePrm) SetIteration(iter uint32) {
x.cliPrm.SetIteration(iter)
}
// SetTrust sets the current global trust value computed at the iteration.
func (x *AnnounceIntermediatePrm) SetTrust(t reputation.PeerToPeerTrust) {
x.cliPrm.SetCurrentValue(t)
}
// AnnounceIntermediateRes groups the resulting values of AnnounceIntermediate operation.
type AnnounceIntermediateRes struct{}
// AnnounceIntermediate sends the global trust value calculated at the specified iteration
// and epoch to to the remote node.
//
// Client, context and key must be set.
//
// Returns any error which prevented the operation from completing correctly in error return.
func AnnounceIntermediate(prm AnnounceIntermediatePrm) (res AnnounceIntermediateRes, err error) {
var cliRes *client.ResAnnounceIntermediateTrust
cliRes, err = prm.cli.AnnounceIntermediateTrust(prm.ctx, prm.cliPrm)
if err == nil {
// pull out an error from status
err = apistatus.ErrFromStatus(cliRes.Status())
}
return
}

View file

@ -0,0 +1,11 @@
// Package internal provides functionality for NeoFS Node Reputation system communication with NeoFS network.
// The base client for accessing remote nodes via NeoFS API is a NeoFS SDK Go API client.
// However, although it encapsulates a useful piece of business logic (e.g. the signature mechanism),
// the Reputation service does not fully use the client's flexible interface.
//
// In this regard, this package provides functions over base API client necessary for the application.
// This allows you to concentrate the entire spectrum of the client's use in one place (this will be convenient
// both when updating the base client and for evaluating the UX of SDK library). So, it is expected that all
// Reputation service packages will be limited to this package for the development of functionality requiring
// NeoFS API communication.
package internal

View file

@ -0,0 +1,112 @@
package local
import (
"crypto/ecdsa"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/reputation/common"
internalclient "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/reputation/internal/client"
coreclient "github.com/TrueCloudLab/frostfs-node/pkg/core/client"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation"
reputationcommon "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/common"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
reputationapi "github.com/TrueCloudLab/frostfs-sdk-go/reputation"
"go.uber.org/zap"
)
// RemoteProviderPrm groups the required parameters of the RemoteProvider's constructor.
//
// All values must comply with the requirements imposed on them.
// Passing incorrect parameter values will result in constructor
// failure (error or panic depending on the implementation).
type RemoteProviderPrm struct {
Key *ecdsa.PrivateKey
Log *logger.Logger
}
// NewRemoteProvider creates a new instance of the RemoteProvider.
//
// Panics if at least one value of the parameters is invalid.
//
// The created RemoteProvider does not require additional
// initialization and is completely ready for work.
func NewRemoteProvider(prm RemoteProviderPrm) *RemoteProvider {
switch {
case prm.Key == nil:
common.PanicOnPrmValue("NetMapSource", prm.Key)
case prm.Log == nil:
common.PanicOnPrmValue("Logger", prm.Log)
}
return &RemoteProvider{
key: prm.Key,
log: prm.Log,
}
}
// RemoteProvider is an implementation of the clientKeyRemoteProvider interface.
type RemoteProvider struct {
key *ecdsa.PrivateKey
log *logger.Logger
}
func (rp RemoteProvider) WithClient(c coreclient.Client) reputationcommon.WriterProvider {
return &TrustWriterProvider{
client: c,
key: rp.key,
log: rp.log,
}
}
type TrustWriterProvider struct {
client coreclient.Client
key *ecdsa.PrivateKey
log *logger.Logger
}
func (twp *TrustWriterProvider) InitWriter(ctx reputationcommon.Context) (reputationcommon.Writer, error) {
return &RemoteTrustWriter{
ctx: ctx,
client: twp.client,
key: twp.key,
log: twp.log,
}, nil
}
type RemoteTrustWriter struct {
ctx reputationcommon.Context
client coreclient.Client
key *ecdsa.PrivateKey
log *logger.Logger
buf []reputationapi.Trust
}
func (rtp *RemoteTrustWriter) Write(t reputation.Trust) error {
var apiTrust reputationapi.Trust
apiTrust.SetValue(t.Value().Float64())
apiTrust.SetPeer(t.Peer())
rtp.buf = append(rtp.buf, apiTrust)
return nil
}
func (rtp *RemoteTrustWriter) Close() error {
epoch := rtp.ctx.Epoch()
rtp.log.Debug("announcing trusts",
zap.Uint64("epoch", epoch),
)
var prm internalclient.AnnounceLocalPrm
prm.SetContext(rtp.ctx)
prm.SetClient(rtp.client)
prm.SetEpoch(epoch)
prm.SetTrusts(rtp.buf)
_, err := internalclient.AnnounceLocal(prm)
return err
}

View file

@ -0,0 +1,107 @@
package local
import (
"bytes"
"errors"
netmapcore "github.com/TrueCloudLab/frostfs-node/pkg/core/netmap"
"github.com/TrueCloudLab/frostfs-node/pkg/services/reputation"
reputationcommon "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/common"
trustcontroller "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/local/controller"
truststorage "github.com/TrueCloudLab/frostfs-node/pkg/services/reputation/local/storage"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
apireputation "github.com/TrueCloudLab/frostfs-sdk-go/reputation"
"go.uber.org/zap"
)
type TrustStorage struct {
Log *logger.Logger
Storage *truststorage.Storage
NmSrc netmapcore.Source
LocalKey []byte
}
func (s *TrustStorage) InitIterator(ctx reputationcommon.Context) (trustcontroller.Iterator, error) {
epoch := ctx.Epoch()
s.Log.Debug("initializing iterator over trusts",
zap.Uint64("epoch", epoch),
)
epochStorage, err := s.Storage.DataForEpoch(epoch)
if err != nil && !errors.Is(err, truststorage.ErrNoPositiveTrust) {
return nil, err
}
return &TrustIterator{
ctx: ctx,
storage: s,
epochStorage: epochStorage,
}, nil
}
type TrustIterator struct {
ctx reputationcommon.Context
storage *TrustStorage
epochStorage *truststorage.EpochTrustValueStorage
}
func (it *TrustIterator) Iterate(h reputation.TrustHandler) error {
if it.epochStorage != nil {
err := it.epochStorage.Iterate(h)
if !errors.Is(err, truststorage.ErrNoPositiveTrust) {
return err
}
}
nm, err := it.storage.NmSrc.GetNetMapByEpoch(it.ctx.Epoch())
if err != nil {
return err
}
// find out if local node is presented in netmap
localIndex := -1
nmNodes := nm.Nodes()
for i := range nmNodes {
if bytes.Equal(nmNodes[i].PublicKey(), it.storage.LocalKey) {
localIndex = i
break
}
}
ln := len(nmNodes)
if localIndex >= 0 && ln > 0 {
ln--
}
// calculate Pj http://ilpubs.stanford.edu:8090/562/1/2002-56.pdf Chapter 4.5.
p := reputation.TrustOne.Div(reputation.TrustValueFromInt(ln))
for i := range nmNodes {
if i == localIndex {
continue
}
var trusted, trusting apireputation.PeerID
trusted.SetPublicKey(nmNodes[i].PublicKey())
trusting.SetPublicKey(it.storage.LocalKey)
trust := reputation.Trust{}
trust.SetPeer(trusted)
trust.SetValue(p)
trust.SetTrustingPeer(trusting)
if err := h(trust); err != nil {
return err
}
}
return nil
}

View file

@ -0,0 +1,90 @@
package ticker
import (
"fmt"
"sync"
)
// IterationHandler is a callback of a certain block advance.
type IterationHandler func()
// IterationsTicker represents a fixed tick number block timer.
//
// It can tick the blocks and perform certain actions
// on block time intervals.
type IterationsTicker struct {
m sync.Mutex
curr uint64
period uint64
times uint64
h IterationHandler
}
// NewIterationsTicker creates a new IterationsTicker.
//
// It guaranties that a handler would be called the
// specified amount of times in the specified amount
// of blocks. After the last meaningful Tick, IterationsTicker
// becomes no-op timer.
//
// Returns an error only if times is greater than totalBlocks.
func NewIterationsTicker(totalBlocks uint64, times uint64, h IterationHandler) (*IterationsTicker, error) {
period := totalBlocks / times
if period == 0 {
return nil, fmt.Errorf("impossible to tick %d times in %d blocks",
times, totalBlocks,
)
}
var curr uint64
// try to make handler calls as rare as possible
if totalBlocks%times != 0 {
extraBlocks := (period+1)*times - totalBlocks
if period >= extraBlocks {
curr = extraBlocks + (period-extraBlocks)/2
period++
}
}
return &IterationsTicker{
curr: curr,
period: period,
times: times,
h: h,
}, nil
}
// Tick ticks one block in the IterationsTicker.
//
// Returns `false` if the timer has finished its operations
// and there will be no more handler calls.
// Calling Tick after the returned `false` is safe, no-op
// and also returns `false`.
func (ft *IterationsTicker) Tick() bool {
ft.m.Lock()
defer ft.m.Unlock()
if ft.times == 0 {
return false
}
ft.curr++
if ft.curr%ft.period == 0 {
ft.h()
ft.times--
if ft.times == 0 {
return false
}
}
return true
}

View file

@ -0,0 +1,118 @@
package ticker
import (
"errors"
"fmt"
"testing"
"github.com/stretchr/testify/require"
)
func TestFixedTimer_Tick(t *testing.T) {
tests := [...]struct {
duration uint64
times uint64
err error
}{
{
duration: 20,
times: 4,
err: nil,
},
{
duration: 6,
times: 6,
err: nil,
},
{
duration: 10,
times: 6,
err: nil,
},
{
duration: 5,
times: 6,
err: errors.New("impossible to tick 6 times in 5 blocks"),
},
}
for _, test := range tests {
t.Run(fmt.Sprintf("duration:%d,times:%d", test.duration, test.times), func(t *testing.T) {
counter := uint64(0)
timer, err := NewIterationsTicker(test.duration, test.times, func() {
counter++
})
if test.err != nil {
require.EqualError(t, err, test.err.Error())
return
}
require.NoError(t, err)
for i := 0; i < int(test.duration); i++ {
if !timer.Tick() {
break
}
}
require.Equal(t, false, timer.Tick())
require.Equal(t, test.times, counter)
})
}
}
func TestFixedTimer_RareCalls(t *testing.T) {
tests := [...]struct {
duration uint64
times uint64
firstCall uint64
period uint64
}{
{
duration: 11,
times: 6,
firstCall: 1,
period: 2,
},
{
duration: 11,
times: 4,
firstCall: 2,
period: 3,
},
{
duration: 20,
times: 3,
firstCall: 4,
period: 7,
},
}
for _, test := range tests {
t.Run(fmt.Sprintf("duration:%d,times:%d", test.duration, test.times), func(t *testing.T) {
var counter uint64
timer, err := NewIterationsTicker(test.duration, test.times, func() {
counter++
})
require.NoError(t, err)
checked := false
for i := 1; i <= int(test.duration); i++ {
if !timer.Tick() {
break
}
if !checked && counter == 1 {
require.Equal(t, test.firstCall, uint64(i))
checked = true
}
}
require.Equal(t, false, timer.Tick())
require.Equal(t, test.times, counter)
})
}
}

View file

@ -0,0 +1,66 @@
package main
import (
"context"
"fmt"
"time"
"github.com/TrueCloudLab/frostfs-api-go/v2/session"
sessionGRPC "github.com/TrueCloudLab/frostfs-api-go/v2/session/grpc"
nodeconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/node"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/event"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/event/netmap"
sessionTransportGRPC "github.com/TrueCloudLab/frostfs-node/pkg/network/transport/session/grpc"
sessionSvc "github.com/TrueCloudLab/frostfs-node/pkg/services/session"
"github.com/TrueCloudLab/frostfs-node/pkg/services/session/storage"
"github.com/TrueCloudLab/frostfs-node/pkg/services/session/storage/persistent"
"github.com/TrueCloudLab/frostfs-node/pkg/services/session/storage/temporary"
"github.com/TrueCloudLab/frostfs-sdk-go/user"
)
type sessionStorage interface {
Create(ctx context.Context, body *session.CreateRequestBody) (*session.CreateResponseBody, error)
Get(ownerID user.ID, tokenID []byte) *storage.PrivateToken
RemoveOld(epoch uint64)
Close() error
}
func initSessionService(c *cfg) {
if persistentSessionPath := nodeconfig.PersistentSessions(c.appCfg).Path(); persistentSessionPath != "" {
persisessions, err := persistent.NewTokenStore(persistentSessionPath,
persistent.WithLogger(c.log),
persistent.WithTimeout(100*time.Millisecond),
persistent.WithEncryptionKey(&c.key.PrivateKey),
)
if err != nil {
panic(fmt.Errorf("could not create persistent session token storage: %w", err))
}
c.privateTokenStore = persisessions
} else {
c.privateTokenStore = temporary.NewTokenStore()
}
c.onShutdown(func() {
_ = c.privateTokenStore.Close()
})
addNewEpochNotificationHandler(c, func(ev event.Event) {
c.privateTokenStore.RemoveOld(ev.(netmap.NewEpoch).EpochNumber())
})
server := sessionTransportGRPC.New(
sessionSvc.NewSignService(
&c.key.PrivateKey,
sessionSvc.NewResponseService(
sessionSvc.NewExecutionService(c.privateTokenStore, c.log),
c.respSvc,
),
),
)
for _, srv := range c.cfgGRPC.servers {
sessionGRPC.RegisterSessionServiceServer(srv, server)
}
}

View file

@ -0,0 +1,43 @@
package main
import (
"sync"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/reputation/ticker"
)
type eigenTrustTickers struct {
m sync.Mutex
timers map[uint64]*ticker.IterationsTicker
}
func (e *eigenTrustTickers) addEpochTimer(epoch uint64, timer *ticker.IterationsTicker) {
e.m.Lock()
defer e.m.Unlock()
e.timers[epoch] = timer
}
func (e *eigenTrustTickers) tick() {
e.m.Lock()
defer e.m.Unlock()
for epoch, t := range e.timers {
if !t.Tick() {
delete(e.timers, epoch)
}
}
}
func tickBlockTimers(c *cfg) {
c.cfgMorph.eigenTrustTicker.tick()
}
func newEigenTrustIterTimer(c *cfg) {
c.cfgMorph.eigenTrustTicker = &eigenTrustTickers{
// it is expected to have max 2 concurrent epoch
// in normal mode work
timers: make(map[uint64]*ticker.IterationsTicker, 2),
}
}

87
cmd/frostfs-node/tree.go Normal file
View file

@ -0,0 +1,87 @@
package main
import (
"context"
"errors"
treeconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/tree"
"github.com/TrueCloudLab/frostfs-node/pkg/core/container"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/pilorama"
containerClient "github.com/TrueCloudLab/frostfs-node/pkg/morph/client/container"
"github.com/TrueCloudLab/frostfs-node/pkg/morph/event"
containerEvent "github.com/TrueCloudLab/frostfs-node/pkg/morph/event/container"
"github.com/TrueCloudLab/frostfs-node/pkg/services/tree"
cid "github.com/TrueCloudLab/frostfs-sdk-go/container/id"
"go.uber.org/zap"
)
type cnrSource struct {
// cache of raw client.
src container.Source
// raw client; no need to cache request results
// since sync is performed once in epoch and is
// expected to receive different results every
// call.
cli *containerClient.Client
}
func (c cnrSource) Get(id cid.ID) (*container.Container, error) {
return c.src.Get(id)
}
func (c cnrSource) List() ([]cid.ID, error) {
return c.cli.List(nil)
}
func initTreeService(c *cfg) {
treeConfig := treeconfig.Tree(c.appCfg)
if !treeConfig.Enabled() {
c.log.Info("tree service is not enabled, skip initialization")
return
}
c.treeService = tree.New(
tree.WithContainerSource(cnrSource{
src: c.cfgObject.cnrSource,
cli: c.shared.cnrClient,
}),
tree.WithEACLSource(c.cfgObject.eaclSource),
tree.WithNetmapSource(c.netMapSource),
tree.WithPrivateKey(&c.key.PrivateKey),
tree.WithLogger(c.log),
tree.WithStorage(c.cfgObject.cfgLocalStorage.localStorage),
tree.WithContainerCacheSize(treeConfig.CacheSize()),
tree.WithReplicationChannelCapacity(treeConfig.ReplicationChannelCapacity()),
tree.WithReplicationWorkerCount(treeConfig.ReplicationWorkerCount()))
for _, srv := range c.cfgGRPC.servers {
tree.RegisterTreeServiceServer(srv, c.treeService)
}
c.workers = append(c.workers, newWorkerFromFunc(func(ctx context.Context) {
c.treeService.Start(ctx)
}))
addNewEpochNotificationHandler(c, func(_ event.Event) {
err := c.treeService.SynchronizeAll()
if err != nil {
c.log.Error("could not synchronize Tree Service", zap.Error(err))
}
})
subscribeToContainerRemoval(c, func(e event.Event) {
ev := e.(containerEvent.DeleteSuccess)
// This is executed asynchronously, so we don't care about the operation taking some time.
c.log.Debug("removing all trees for container", zap.Stringer("cid", ev.ID))
err := c.treeService.DropTree(context.Background(), ev.ID, "")
if err != nil && !errors.Is(err, pilorama.ErrTreeNotFound) {
// Ignore pilorama.ErrTreeNotFound but other errors, including shard.ErrReadOnly, should be logged.
c.log.Error("container removal event received, but trees weren't removed",
zap.Stringer("cid", ev.ID),
zap.String("error", err.Error()))
}
})
c.onShutdown(c.treeService.Shutdown)
}

View file

@ -0,0 +1,104 @@
package main
import (
"fmt"
"path/filepath"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
engineconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine"
shardconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard"
loggerconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/logger"
treeconfig "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/tree"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/blobovniczatree"
"github.com/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/fstree"
"github.com/TrueCloudLab/frostfs-node/pkg/util/logger"
)
// validateConfig validates storage node configuration.
func validateConfig(c *config.Config) error {
// logger configuration validation
var loggerPrm logger.Prm
err := loggerPrm.SetLevelString(loggerconfig.Level(c))
if err != nil {
return fmt.Errorf("invalid logger level: %w", err)
}
// shard configuration validation
shardNum := 0
paths := make(map[string]pathDescription)
return engineconfig.IterateShards(c, false, func(sc *shardconfig.Config) error {
if sc.WriteCache().Enabled() {
err := addPath(paths, "writecache", shardNum, sc.WriteCache().Path())
if err != nil {
return err
}
}
if err := addPath(paths, "metabase", shardNum, sc.Metabase().Path()); err != nil {
return err
}
treeConfig := treeconfig.Tree(c)
if treeConfig.Enabled() {
err := addPath(paths, "pilorama", shardNum, sc.Pilorama().Path())
if err != nil {
return err
}
}
blobstor := sc.BlobStor().Storages()
if len(blobstor) != 2 {
// TODO (@fyrcik): remove after #1522
return fmt.Errorf("blobstor section must have 2 components, got: %d", len(blobstor))
}
for i := range blobstor {
switch blobstor[i].Type() {
case fstree.Type, blobovniczatree.Type:
default:
// FIXME #1764 (@fyrchik): this line is currently unreachable,
// because we panic in `sc.BlobStor().Storages()`.
return fmt.Errorf("unexpected storage type: %s (shard %d)",
blobstor[i].Type(), shardNum)
}
if blobstor[i].Perm()&0600 != 0600 {
return fmt.Errorf("invalid permissions for blobstor component: %s, "+
"expected at least rw- for the owner (shard %d)",
blobstor[i].Perm(), shardNum)
}
if blobstor[i].Path() == "" {
return fmt.Errorf("blobstor component path is empty (shard %d)", shardNum)
}
err := addPath(paths, fmt.Sprintf("blobstor[%d]", i), shardNum, blobstor[i].Path())
if err != nil {
return err
}
}
shardNum++
return nil
})
}
type pathDescription struct {
shard int
component string
}
func addPath(paths map[string]pathDescription, component string, shard int, path string) error {
if path == "" {
return fmt.Errorf("%s at shard %d has empty path", component, shard)
}
path = filepath.Clean(path)
c, ok := paths[path]
if ok {
return fmt.Errorf("%s at shard %d and %s at shard %d have the same paths: %s",
c.component, c.shard, component, shard, path)
}
paths[path] = pathDescription{shard: shard, component: component}
return nil
}

View file

@ -0,0 +1,38 @@
package main
import (
"os"
"path/filepath"
"testing"
"github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
configtest "github.com/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"github.com/stretchr/testify/require"
)
func TestValidate(t *testing.T) {
const exampleConfigPrefix = "../../config/"
t.Run("examples", func(t *testing.T) {
p := filepath.Join(exampleConfigPrefix, "example/node")
configtest.ForEachFileType(p, func(c *config.Config) {
var err error
require.NotPanics(t, func() {
err = validateConfig(c)
})
require.NoError(t, err)
})
})
t.Run("mainnet", func(t *testing.T) {
os.Clearenv() // ENVs have priority over config files, so we do this in tests
p := filepath.Join(exampleConfigPrefix, "mainnet/config.yml")
c := config.New(config.Prm{}, config.WithConfigFile(p))
require.NoError(t, validateConfig(c))
})
t.Run("testnet", func(t *testing.T) {
os.Clearenv() // ENVs have priority over config files, so we do this in tests
p := filepath.Join(exampleConfigPrefix, "testnet/config.yml")
c := config.New(config.Prm{}, config.WithConfigFile(p))
require.NoError(t, validateConfig(c))
})
}

View file

@ -0,0 +1,34 @@
package main
import (
"context"
)
type worker interface {
Run(context.Context)
}
type workerFromFunc struct {
fn func(context.Context)
}
func newWorkerFromFunc(fn func(ctx context.Context)) worker {
return &workerFromFunc{
fn: fn,
}
}
func (w *workerFromFunc) Run(ctx context.Context) {
w.fn(ctx)
}
func startWorkers(c *cfg) {
for _, wrk := range c.workers {
c.wg.Add(1)
go func(w worker) {
w.Run(c.ctx)
c.wg.Done()
}(wrk)
}
}