[#411] Remove unnecessary pointers for sync objects

Signed-off-by: Alejandro Lopez <a.lopez@yadro.com>
fix/nil-innerring-metrics-panic
Alejandro Lopez 2023-05-30 10:14:37 +03:00 committed by Evgenii Stratonikov
parent f934abed8f
commit faca861451
20 changed files with 39 additions and 64 deletions

View File

@ -25,19 +25,18 @@ type valueWithTime[V any] struct {
} }
type locker struct { type locker struct {
mtx *sync.Mutex mtx sync.Mutex
waiters int // not protected by mtx, must used outer mutex to update concurrently waiters int // not protected by mtx, must used outer mutex to update concurrently
} }
type keyLocker[K comparable] struct { type keyLocker[K comparable] struct {
lockers map[K]*locker lockers map[K]*locker
lockersMtx *sync.Mutex lockersMtx sync.Mutex
} }
func newKeyLocker[K comparable]() *keyLocker[K] { func newKeyLocker[K comparable]() *keyLocker[K] {
return &keyLocker[K]{ return &keyLocker[K]{
lockers: make(map[K]*locker), lockers: make(map[K]*locker),
lockersMtx: &sync.Mutex{},
} }
} }
@ -53,7 +52,6 @@ func (l *keyLocker[K]) LockKey(key K) {
} }
locker := &locker{ locker := &locker{
mtx: &sync.Mutex{},
waiters: 1, waiters: 1,
} }
locker.mtx.Lock() locker.mtx.Lock()

View File

@ -327,7 +327,7 @@ type internals struct {
log *logger.Logger log *logger.Logger
wg *sync.WaitGroup wg sync.WaitGroup
workers []worker workers []worker
closers []closer closers []closer
@ -589,7 +589,6 @@ func initInternals(appCfg *config.Config, log *logger.Logger) internals {
appCfg: appCfg, appCfg: appCfg,
internalErr: make(chan error), internalErr: make(chan error),
log: log, log: log,
wg: new(sync.WaitGroup),
apiVersion: version.Current(), apiVersion: version.Current(),
healthStatus: &healthStatus, healthStatus: &healthStatus,
} }

View File

@ -49,7 +49,7 @@ type (
Processor struct { Processor struct {
parsedWallets []util.Uint160 parsedWallets []util.Uint160
// protects parsedWallets from concurrent change // protects parsedWallets from concurrent change
pwLock *sync.RWMutex pwLock sync.RWMutex
log *logger.Logger log *logger.Logger
metrics metrics.Register metrics metrics.Register
pool *ants.Pool pool *ants.Pool
@ -99,7 +99,6 @@ func New(p *Params) (*Processor, error) {
return &Processor{ return &Processor{
parsedWallets: p.ParsedWallets, parsedWallets: p.ParsedWallets,
pwLock: new(sync.RWMutex),
log: p.Log, log: p.Log,
metrics: metricsRegister, metrics: metricsRegister,
pool: pool, pool: pool,

View File

@ -68,7 +68,7 @@ type (
epochState EpochState epochState EpochState
alphabetState AlphabetState alphabetState AlphabetState
converter PrecisionConverter converter PrecisionConverter
mintEmitLock *sync.Mutex mintEmitLock sync.Mutex
mintEmitCache *lru.Cache[string, uint64] mintEmitCache *lru.Cache[string, uint64]
mintEmitThreshold uint64 mintEmitThreshold uint64
mintEmitValue fixedn.Fixed8 mintEmitValue fixedn.Fixed8
@ -148,7 +148,6 @@ func New(p *Params) (*Processor, error) {
epochState: p.EpochState, epochState: p.EpochState,
alphabetState: p.AlphabetState, alphabetState: p.AlphabetState,
converter: p.Converter, converter: p.Converter,
mintEmitLock: new(sync.Mutex),
mintEmitCache: lruCache, mintEmitCache: lruCache,
mintEmitThreshold: p.MintEmitThreshold, mintEmitThreshold: p.MintEmitThreshold,
mintEmitValue: p.MintEmitValue, mintEmitValue: p.MintEmitValue,

View File

@ -9,7 +9,7 @@ import (
type ( type (
cleanupTable struct { cleanupTable struct {
*sync.RWMutex sync.RWMutex
enabled bool enabled bool
threshold uint64 threshold uint64
lastAccess map[string]epochStampWithNodeInfo lastAccess map[string]epochStampWithNodeInfo
@ -29,7 +29,6 @@ type (
func newCleanupTable(enabled bool, threshold uint64) cleanupTable { func newCleanupTable(enabled bool, threshold uint64) cleanupTable {
return cleanupTable{ return cleanupTable{
RWMutex: new(sync.RWMutex),
enabled: enabled, enabled: enabled,
threshold: threshold, threshold: threshold,
lastAccess: make(map[string]epochStampWithNodeInfo), lastAccess: make(map[string]epochStampWithNodeInfo),

View File

@ -20,7 +20,7 @@ type StorageEngine struct {
removeDuplicatesInProgress atomic.Bool removeDuplicatesInProgress atomic.Bool
mtx *sync.RWMutex mtx sync.RWMutex
shards map[string]hashedShard shards map[string]hashedShard
@ -225,15 +225,12 @@ func New(opts ...Option) *StorageEngine {
} }
return &StorageEngine{ return &StorageEngine{
cfg: c, cfg: c,
mtx: new(sync.RWMutex), shards: make(map[string]hashedShard),
shards: make(map[string]hashedShard), shardPools: make(map[string]util.WorkerPool),
shardPools: make(map[string]util.WorkerPool), closeCh: make(chan struct{}),
closeCh: make(chan struct{}), setModeCh: make(chan setModeRequest),
setModeCh: make(chan setModeRequest), evacuateLimiter: &evacuationLimiter{},
evacuateLimiter: &evacuationLimiter{
guard: &sync.RWMutex{},
},
} }
} }

View File

@ -113,7 +113,7 @@ type evacuationLimiter struct {
eg *errgroup.Group eg *errgroup.Group
cancel context.CancelFunc cancel context.CancelFunc
guard *sync.RWMutex guard sync.RWMutex
} }
func (l *evacuationLimiter) TryStart(ctx context.Context, shardIDs []string, result *EvacuateShardRes) (*errgroup.Group, context.Context, error) { func (l *evacuationLimiter) TryStart(ctx context.Context, shardIDs []string, result *EvacuateShardRes) (*errgroup.Group, context.Context, error) {

View File

@ -66,7 +66,7 @@ type Client struct {
// switchLock protects endpoints, inactive, and subscription-related fields. // switchLock protects endpoints, inactive, and subscription-related fields.
// It is taken exclusively during endpoint switch and locked in shared mode // It is taken exclusively during endpoint switch and locked in shared mode
// on every normal call. // on every normal call.
switchLock *sync.RWMutex switchLock sync.RWMutex
// channel for internal stop // channel for internal stop
closeChan chan struct{} closeChan chan struct{}
@ -83,7 +83,7 @@ type Client struct {
} }
type cache struct { type cache struct {
m *sync.RWMutex m sync.RWMutex
nnsHash *util.Uint160 nnsHash *util.Uint160
gKey *keys.PublicKey gKey *keys.PublicKey

View File

@ -4,7 +4,6 @@ import (
"context" "context"
"errors" "errors"
"fmt" "fmt"
"sync"
"time" "time"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs" "git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
@ -105,13 +104,12 @@ func New(ctx context.Context, key *keys.PrivateKey, opts ...Option) (*Client, er
} }
cli := &Client{ cli := &Client{
cache: newClientCache(), cache: newClientCache(),
logger: cfg.logger, logger: cfg.logger,
acc: acc, acc: acc,
accAddr: accAddr, accAddr: accAddr,
cfg: *cfg, cfg: *cfg,
switchLock: &sync.RWMutex{}, closeChan: make(chan struct{}),
closeChan: make(chan struct{}),
} }
cli.endpoints.init(cfg.endpoints) cli.endpoints.init(cfg.endpoints)
@ -198,7 +196,6 @@ func newActor(ws *rpcclient.WSClient, acc *wallet.Account, cfg cfg) (*actor.Acto
func newClientCache() cache { func newClientCache() cache {
c, _ := lru.New[util.Uint256, uint32](100) // returns error only if size is negative c, _ := lru.New[util.Uint256, uint32](100) // returns error only if size is negative
return cache{ return cache{
m: &sync.RWMutex{},
txHeights: c, txHeights: c,
} }
} }

View File

@ -42,7 +42,7 @@ type (
} }
subscriber struct { subscriber struct {
*sync.RWMutex sync.RWMutex
log *logger.Logger log *logger.Logger
client *client.Client client *client.Client
@ -163,7 +163,6 @@ func New(ctx context.Context, p *Params) (Subscriber, error) {
} }
sub := &subscriber{ sub := &subscriber{
RWMutex: new(sync.RWMutex),
log: p.Log, log: p.Log,
client: p.Client, client: p.Client,
notifyChan: make(chan *state.ContainedNotificationEvent), notifyChan: make(chan *state.ContainedNotificationEvent),

View File

@ -17,7 +17,7 @@ type BlockTickHandler func()
type BlockTimer struct { type BlockTimer struct {
rolledBack bool rolledBack bool
mtx *sync.Mutex mtx sync.Mutex
dur BlockMeter dur BlockMeter
@ -64,7 +64,6 @@ func StaticBlockMeter(d uint32) BlockMeter {
// Reset should be called before timer ticking. // Reset should be called before timer ticking.
func NewBlockTimer(dur BlockMeter, h BlockTickHandler) *BlockTimer { func NewBlockTimer(dur BlockMeter, h BlockTickHandler) *BlockTimer {
return &BlockTimer{ return &BlockTimer{
mtx: new(sync.Mutex),
dur: dur, dur: dur,
mul: 1, mul: 1,
div: 1, div: 1,
@ -80,7 +79,6 @@ func NewBlockTimer(dur BlockMeter, h BlockTickHandler) *BlockTimer {
// Do not use delta handlers with pulse in this timer. // Do not use delta handlers with pulse in this timer.
func NewOneTickTimer(dur BlockMeter, h BlockTickHandler) *BlockTimer { func NewOneTickTimer(dur BlockMeter, h BlockTickHandler) *BlockTimer {
return &BlockTimer{ return &BlockTimer{
mtx: new(sync.Mutex),
dur: dur, dur: dur,
mul: 1, mul: 1,
div: 1, div: 1,

View File

@ -25,7 +25,7 @@ type Writer struct {
js nats.JetStreamContext js nats.JetStreamContext
nc *nats.Conn nc *nats.Conn
m *sync.RWMutex m sync.RWMutex
createdStreams map[string]struct{} createdStreams map[string]struct{}
opts opts
} }
@ -84,7 +84,6 @@ func (n *Writer) Notify(topic string, address oid.Address) error {
// New creates new Writer. // New creates new Writer.
func New(oo ...Option) *Writer { func New(oo ...Option) *Writer {
w := &Writer{ w := &Writer{
m: &sync.RWMutex{},
createdStreams: make(map[string]struct{}), createdStreams: make(map[string]struct{}),
opts: opts{ opts: opts{
log: &logger.Logger{Logger: zap.L()}, log: &logger.Logger{Logger: zap.L()},

View File

@ -23,8 +23,8 @@ import (
) )
type getRequestForwarder struct { type getRequestForwarder struct {
OnceResign *sync.Once OnceResign sync.Once
OnceHeaderSending *sync.Once OnceHeaderSending sync.Once
GlobalProgress int GlobalProgress int
Key *ecdsa.PrivateKey Key *ecdsa.PrivateKey
Request *objectV2.GetRequest Request *objectV2.GetRequest

View File

@ -23,7 +23,7 @@ import (
) )
type getRangeRequestForwarder struct { type getRangeRequestForwarder struct {
OnceResign *sync.Once OnceResign sync.Once
GlobalProgress int GlobalProgress int
Key *ecdsa.PrivateKey Key *ecdsa.PrivateKey
Request *objectV2.GetRangeRequest Request *objectV2.GetRangeRequest

View File

@ -25,7 +25,7 @@ import (
type headRequestForwarder struct { type headRequestForwarder struct {
Request *objectV2.HeadRequest Request *objectV2.HeadRequest
Response *objectV2.HeadResponse Response *objectV2.HeadResponse
OnceResign *sync.Once OnceResign sync.Once
ObjectAddr oid.Address ObjectAddr oid.Address
Key *ecdsa.PrivateKey Key *ecdsa.PrivateKey
} }

View File

@ -5,7 +5,6 @@ import (
"crypto/sha256" "crypto/sha256"
"errors" "errors"
"hash" "hash"
"sync"
objectV2 "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/object" objectV2 "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/object"
"git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/refs" "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/refs"
@ -59,12 +58,10 @@ func (s *Service) toPrm(req *objectV2.GetRequest, stream objectSvc.GetObjectStre
} }
forwarder := &getRequestForwarder{ forwarder := &getRequestForwarder{
OnceResign: &sync.Once{}, GlobalProgress: 0,
OnceHeaderSending: &sync.Once{}, Key: key,
GlobalProgress: 0, Request: req,
Key: key, Stream: streamWrapper,
Request: req,
Stream: streamWrapper,
} }
p.SetRequestForwarder(groupAddressRequestForwarder(forwarder.forwardRequestToNode)) p.SetRequestForwarder(groupAddressRequestForwarder(forwarder.forwardRequestToNode))
@ -115,7 +112,6 @@ func (s *Service) toRangePrm(req *objectV2.GetRangeRequest, stream objectSvc.Get
} }
forwarder := &getRangeRequestForwarder{ forwarder := &getRangeRequestForwarder{
OnceResign: &sync.Once{},
GlobalProgress: 0, GlobalProgress: 0,
Key: key, Key: key,
Request: req, Request: req,
@ -254,7 +250,6 @@ func (s *Service) toHeadPrm(req *objectV2.HeadRequest, resp *objectV2.HeadRespon
forwarder := &headRequestForwarder{ forwarder := &headRequestForwarder{
Request: req, Request: req,
Response: resp, Response: resp,
OnceResign: &sync.Once{},
ObjectAddr: objAddr, ObjectAddr: objAddr,
Key: key, Key: key,
} }

View File

@ -20,7 +20,7 @@ import (
) )
type requestForwarder struct { type requestForwarder struct {
OnceResign *sync.Once OnceResign sync.Once
Request *objectV2.SearchRequest Request *objectV2.SearchRequest
Key *ecdsa.PrivateKey Key *ecdsa.PrivateKey
} }

View File

@ -4,7 +4,6 @@ import (
"context" "context"
"errors" "errors"
"fmt" "fmt"
"sync"
objectV2 "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/object" objectV2 "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/client" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/client"
@ -51,9 +50,8 @@ func (s *Service) toPrm(req *objectV2.SearchRequest, stream objectSvc.SearchStre
} }
forwarder := &requestForwarder{ forwarder := &requestForwarder{
OnceResign: &sync.Once{}, Request: req,
Request: req, Key: key,
Key: key,
} }
p.SetRequestForwarder(groupAddressRequestForwarder(forwarder.forwardRequest)) p.SetRequestForwarder(groupAddressRequestForwarder(forwarder.forwardRequest))

View File

@ -29,7 +29,7 @@ type Option func(*cfg)
// Traverser represents utility for controlling // Traverser represents utility for controlling
// traversal of object placement vectors. // traversal of object placement vectors.
type Traverser struct { type Traverser struct {
mtx *sync.RWMutex mtx sync.RWMutex
vectors [][]netmap.NodeInfo vectors [][]netmap.NodeInfo
@ -107,7 +107,6 @@ func NewTraverser(opts ...Option) (*Traverser, error) {
} }
return &Traverser{ return &Traverser{
mtx: new(sync.RWMutex),
rem: rem, rem: rem,
vectors: ns, vectors: ns,
}, nil }, nil

View File

@ -18,7 +18,7 @@ type key struct {
// expiring (removing) session tokens. // expiring (removing) session tokens.
// Must be created only via calling NewTokenStore. // Must be created only via calling NewTokenStore.
type TokenStore struct { type TokenStore struct {
mtx *sync.RWMutex mtx sync.RWMutex
tokens map[key]*storage.PrivateToken tokens map[key]*storage.PrivateToken
} }
@ -28,7 +28,6 @@ type TokenStore struct {
// The elements of the instance are stored in the map. // The elements of the instance are stored in the map.
func NewTokenStore() *TokenStore { func NewTokenStore() *TokenStore {
return &TokenStore{ return &TokenStore{
mtx: new(sync.RWMutex),
tokens: make(map[key]*storage.PrivateToken), tokens: make(map[key]*storage.PrivateToken),
} }
} }