[#561] writecache: Make badger keys counting async #613

Closed
aarifullin wants to merge 1 commit from aarifullin/frostfs-node:feature/561-init_count into master
9 changed files with 291 additions and 23 deletions

View file

@ -286,6 +286,9 @@ const (
ShardCouldNotMarkObjectToDeleteInMetabase = "could not mark object to delete in metabase" ShardCouldNotMarkObjectToDeleteInMetabase = "could not mark object to delete in metabase"
WritecacheBadgerInitExperimental = "initializing badger-backed experimental writecache" WritecacheBadgerInitExperimental = "initializing badger-backed experimental writecache"
WritecacheTriedToFlushItemsFromWritecache = "tried to flush items from write-cache" WritecacheTriedToFlushItemsFromWritecache = "tried to flush items from write-cache"
WritecacheBadgerCouldNotCountKeys = "could not count keys from badger by iteration"
WritecacheBadgerCouldNotFlushKeysCount = "could not flush keys count to badger"
WritecacheBadgerCouldNotDeleteKeysCount = "could not delete keys count from badger"
WritecacheWaitingForChannelsToFlush = "waiting for channels to flush" WritecacheWaitingForChannelsToFlush = "waiting for channels to flush"
WritecacheFillingFlushMarksForObjectsInFSTree = "filling flush marks for objects in FSTree" WritecacheFillingFlushMarksForObjectsInFSTree = "filling flush marks for objects in FSTree"
WritecacheFinishedUpdatingFSTreeFlushMarks = "finished updating FSTree flush marks" WritecacheFinishedUpdatingFSTreeFlushMarks = "finished updating FSTree flush marks"

View file

@ -1,6 +1,7 @@
package writecachebadger package writecachebadger
import ( import (
"encoding/binary"
"sync" "sync"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs" "git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
@ -25,6 +26,8 @@ type cache struct {
closeCh chan struct{} closeCh chan struct{}
// wg is a wait group for flush workers. // wg is a wait group for flush workers.
wg sync.WaitGroup wg sync.WaitGroup
// initCounterWG is necessary to wait for asynchronous initialization.
initCounterWG sync.WaitGroup
// store contains underlying database. // store contains underlying database.
store store
} }
@ -100,8 +103,29 @@ func (c *cache) Init() error {
return nil return nil
} }
// flushKeysCount writes the keys count to db if it should be flushed down.
func (c *cache) flushKeysCount() {
// Close may happen after resetting mode
if c.db.IsClosed() {
return
}
if c.objCounters.isReadyToFlush() {
k := keyCountPrefix
v := make([]byte, 8)
binary.LittleEndian.PutUint64(v, c.objCounters.DB())
if err := c.putRaw(k[:], v); err != nil {
c.log.Error(logs.WritecacheBadgerCouldNotFlushKeysCount, zap.Error(err))
}
}
}
// Close closes db connection and stops services. Executes ObjectCounters.FlushAndClose op. // Close closes db connection and stops services. Executes ObjectCounters.FlushAndClose op.
func (c *cache) Close() error { func (c *cache) Close() error {
c.cancelInitCountByCloseAndWait()
c.flushKeysCount()
// We cannot lock mutex for the whole operation duration // We cannot lock mutex for the whole operation duration

Is it correct to flush keys if we canceled counter iteration?

Is it correct to flush keys if we canceled counter iteration?

If you look at flushKeysCount() definition then you can see that if objCounters is not ready to be flushed then nothing will happen. I can move these checks out of flushKeysCount

If you look at `flushKeysCount()` definition then you can see that if objCounters is not ready to be flushed then nothing will happen. I can move these checks out of `flushKeysCount`
// because it is taken by some background workers, so `wg.Wait()` is done without modeMtx. // because it is taken by some background workers, so `wg.Wait()` is done without modeMtx.

Do we have a way to cancel this process?

Do we have a way to cancel this process?

I have introduced the wait method that is cancelled by timeout

I have introduced the wait method that is cancelled by timeout

So not we wait for some timeout, but it seems we still don't cancel anything.
Key counting process is completely auxiliary and if we have it, this means we didn't have key count on startup, thus we do not lose anything by canceling it.
It seems wrong to block by any time here.

So not we wait for some timeout, but it seems we still don't cancel anything. Key counting process is completely auxiliary and if we have it, this means we didn't have key count on startup, thus we do not lose anything by canceling it. It seems wrong to block by any time here.

Okay. It seems you have meant

Do we have a way to cancel the counting at the start of Close invocation

I was confused but for now I got your idea. I will make cancellation for counting

Okay. It seems you have meant > Do we have a way to cancel the counting at the start of `Close` invocation I was confused but for now I got your idea. I will make cancellation for counting

@fyrchik
Please, check out how I have implemented soft cancellation

for it.Rewind(); it.Valid(); it.Next() {
	select {
		case <-c.objCounters.cDB.closeCh:
			return errCountCancelled
		case <-c.objCounters.cDB.modeCh:
			return errCountCancelled
		default:
			inDB++
	}
}

And still we need to have c.initCounterWG.Wait() to correctly finish gorotine. You can check explanation for cancelInitCountBySetModeAndWait

@fyrchik Please, check out how I have implemented _soft_ cancellation ```golang for it.Rewind(); it.Valid(); it.Next() { select { case <-c.objCounters.cDB.closeCh: return errCountCancelled case <-c.objCounters.cDB.modeCh: return errCountCancelled default: inDB++ } } ``` And still we need to have `c.initCounterWG.Wait()` to correctly finish gorotine. You can check explanation for `cancelInitCountBySetModeAndWait`
c.modeMtx.Lock() c.modeMtx.Lock()
@ -125,5 +149,13 @@ func (c *cache) Close() error {
} }
} }
c.metrics.Close() c.metrics.Close()
return nil return nil
} }
// cancelInitCountByCloseAndWait cancels the counting process if Close is invoked
// and waits for correct finish of goroutine within initCount.
func (c *cache) cancelInitCountByCloseAndWait() {
c.objCounters.cDB.cancelByClose()
c.initCounterWG.Wait()
}

View file

@ -14,6 +14,7 @@ import (
"github.com/dgraph-io/badger/v4" "github.com/dgraph-io/badger/v4"
"go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace" "go.opentelemetry.io/otel/trace"
"go.uber.org/zap"
) )
// Delete removes object from write-cache. // Delete removes object from write-cache.
@ -68,3 +69,33 @@ func (c *cache) Delete(ctx context.Context, addr oid.Address) error {
return metaerr.Wrap(err) return metaerr.Wrap(err)
} }
// deleteRawKey removes object from write-cache by raw-represented key.
//
// Returns an error of type apistatus.ObjectNotFound if object is missing in write-cache.
func (c *cache) deleteRawKey(key []byte) error {
aarifullin marked this conversation as resolved Outdated

This is duplicated from Delete. I am wondering how this definition can be generilized

This is duplicated from `Delete`. I am wondering how this definition can be generilized
err := c.db.Update(func(tx *badger.Txn) error {
it, err := tx.Get(key)
if err != nil {
if err == badger.ErrKeyNotFound {
return logicerr.Wrap(new(apistatus.ObjectNotFound))
}
return err
}
if it.ValueSize() > 0 {
err := tx.Delete(key)
if err == nil {
storagelog.Write(c.log,
zap.String("key", string(key)),
storagelog.StorageTypeField(wcStorageType),
storagelog.OpField("db DELETE"),
)
c.objCounters.DecDB()
}
return err
}
return nil
})
return metaerr.Wrap(err)
}

View file

@ -93,3 +93,14 @@ func Get(db *badger.DB, key []byte) ([]byte, error) {
return value, metaerr.Wrap(err) return value, metaerr.Wrap(err)
} }
// getRawKey returns raw-data from write-cache.
//
// Returns an error of type apistatus.ObjectNotFound if the requested object is missing in write-cache.
func (c *cache) getRawKey(key []byte) ([]byte, error) {
value, err := Get(c.db, key)
if err != nil {
return nil, err

Is it always NotFound?

Is it always `NotFound`?

Sorry, you're right. The raw key has nothing common with ObjectNotFound

Sorry, you're right. The raw key has nothing common with `ObjectNotFound`
}
return value, nil
}

View file

@ -22,6 +22,8 @@ func (c *cache) SetMode(m mode.Mode) error {
)) ))
defer span.End() defer span.End()
c.cancelInitCountBySetModeAndWait()
c.modeMtx.Lock() c.modeMtx.Lock()
defer c.modeMtx.Unlock() defer c.modeMtx.Unlock()
@ -32,6 +34,13 @@ func (c *cache) SetMode(m mode.Mode) error {
return err return err
} }
// cancelInitCountBySetModeAndWait cancels the counting process if SetMode is invoked
// and waits for correct finish of goroutine within initCount.
func (c *cache) cancelInitCountBySetModeAndWait() {
c.objCounters.cDB.cancelBySetMode()
c.initCounterWG.Wait()
}
// setMode applies new mode. Must be called with cache.modeMtx lock taken. // setMode applies new mode. Must be called with cache.modeMtx lock taken.
func (c *cache) setMode(ctx context.Context, m mode.Mode) error { func (c *cache) setMode(ctx context.Context, m mode.Mode) error {
var err error var err error

View file

@ -34,6 +34,9 @@ type options struct {
metrics writecache.Metrics metrics writecache.Metrics
// gcInterval is the interval duration to run the GC cycle. // gcInterval is the interval duration to run the GC cycle.
gcInterval time.Duration gcInterval time.Duration
// recountKeys indicates if keys must be forcefully counted only by DB iteration. This flag should be
// used if badger contains invalid keys count record.
recountKeys bool
} }
// WithLogger sets logger. // WithLogger sets logger.
@ -108,3 +111,10 @@ func WithGCInterval(d time.Duration) Option {
o.gcInterval = d o.gcInterval = d
} }
} }
// WithRecountKeys sets the recount keys flag.
func WithRecountKeys(recount bool) Option {
return func(o *options) {
o.recountKeys = recount
}
}

View file

@ -8,6 +8,7 @@ import (
storagelog "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/log" storagelog "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/log"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing" "git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
"github.com/dgraph-io/badger/v4"
"go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace" "go.opentelemetry.io/otel/trace"
) )
@ -80,3 +81,25 @@ func (c *cache) put(obj objectInfo) error {
} }
return err return err
} }
// putRaw persists raw-represent key and value to the write-cache
acid-ant marked this conversation as resolved Outdated

pushes the to the flush -> pushes them to the flush?

pushes the to the flush -> pushes them to the flush?
// database and pushes to the flush workers queue.
func (c *cache) putRaw(key, value []byte) error {
cacheSize := c.estimateCacheSize()
if c.maxCacheSize < c.incSizeDB(cacheSize) {
return writecache.ErrOutOfSpace
}
dstepanov-yadro marked this conversation as resolved Outdated

Since this method is used for flush, batch looks redundant.

Since this method is used for flush, batch looks redundant.

To be honest, this definition has been taken from

func (c *cache) put(obj objectInfo) error

above

@ale64bit
Can you tell, please, is the batch really redundant here?

To be honest, this definition has been taken from ```golang func (c *cache) put(obj objectInfo) error ``` above @ale64bit Can you tell, please, is the batch really redundant here?

There's not much point in having a WriteBatch for a single update. Update can be used instead.
This was introduced by me but it doesn't really match what bbolt does internally via the Batch call. Feel free to correct it.

There's not much point in having a `WriteBatch` for a single update. `Update` can be used instead. This was introduced by me but it doesn't really match what bbolt does internally via the `Batch` call. Feel free to correct it.
err := c.db.Update(func(txn *badger.Txn) error {
err := txn.Set(key, value)
return err
})
if err == nil {
storagelog.Write(c.log,
storagelog.StorageTypeField(wcStorageType),
storagelog.OpField("db PUT"),
)
c.objCounters.IncDB()
}
return err
}

View file

@ -1,11 +1,18 @@
package writecachebadger package writecachebadger
import ( import (
"fmt" "encoding/binary"
"math" "errors"
"sync/atomic" "sync"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
"github.com/dgraph-io/badger/v4" "github.com/dgraph-io/badger/v4"
"go.uber.org/atomic"
"go.uber.org/zap"
)
var (
errCountCancelled = errors.New("counting was cancelled")
) )
func (c *cache) estimateCacheSize() uint64 { func (c *cache) estimateCacheSize() uint64 {
aarifullin marked this conversation as resolved
Review

The file size on disk is still used here, but bbolt uses the number of objects. But if we use a counter of objects, it is not clear how asynchronous counting will help us.

The file size on disk is still used here, but bbolt uses the number of objects. But if we use a counter of objects, it is not clear how asynchronous counting will help us.
Review

You're right, we don't need to use DB() if we want roughly estimate size of db data.

But we cannot use EstimateSize if need the exact number of keys in DB:

// EstimateSize can be used to get rough estimate of data size for a given prefix.

So, we won't be able to count the exact number using EstimateSize and the rough approximation for keys number may mislead if we use it for metrics or other purposes

cc @fyrchik @ale64bit

You're right, we don't need to use `DB()` if we want roughly estimate size of db data. But we cannot use `EstimateSize` if need the exact number of keys in DB: ```golang // EstimateSize can be used to get rough estimate of data size for a given prefix. ``` So, we won't be able to count the exact number using `EstimateSize` and the rough approximation for keys number may mislead if we use it for metrics or other purposes cc @fyrchik @ale64bit
Review

But such rough approximation with size rounding is used for bbolt.

For example, imagine that you are a user and you see that writecache is not being used. You open the metrics and see that the limit has not been reached. Or vice versa.

But such rough approximation with size rounding is used for bbolt. For example, imagine that you are a user and you see that writecache is not being used. You open the metrics and see that the limit has not been reached. Or vice versa.
Review

Probably, we talk about different things. Let me clarify:

DB() gives the exact number of keys currently saved in badger.
We cannot use EstimateSize() to calculate the number of keys: keys = EstimateSize() / EstimateKeySize() - this is incorrect

Probably, we talk about different things. Let me clarify: `DB()` gives the exact number of keys currently saved in badger. We cannot use `EstimateSize()` to calculate the number of keys: `keys = EstimateSize() / EstimateKeySize()` - this is incorrect
@ -18,40 +25,180 @@ func (c *cache) incSizeDB(sz uint64) uint64 {
return sz + c.maxObjectSize return sz + c.maxObjectSize
} }
type countAsync struct {
value atomic.Uint64
// mutable checks whether the value can be incremented/decremented.
mutable bool
// mutMtx is mutex for mutable flag.
mutMtx sync.RWMutex

We have a mutex for a single primitive variable, why not just use atomics? (a question, not a suggestion)
Mutex could give stronger guarante, but I dont see them being taken for the whole operation duration (currently we have a race condition, but possibly do not exploit it)

We have a mutex for a single primitive variable, why not just use atomics? (a question, not a suggestion) Mutex could give stronger guarante, but I dont see them being taken for the whole operation duration (currently we have a race condition, but possibly do not exploit it)
// init checks whether the value is finally initialized.
init bool
// initMtx is mutex for init flag.
initMtx sync.RWMutex
// modeCh is channel to indicate SetMode has been invoked and
// counting must be cancelled.
aarifullin marked this conversation as resolved Outdated

So new operations will hang until counters are finished?

So new operations _will_ hang until counters are finished?

counters are finished

Until db transaction starts :)

> counters are finished Until db transaction starts :)

Removed this waitGroup and replaced it with isMutable check

Removed this waitGroup and replaced it with `isMutable` check

P.S. I am going to resolve the conversation to make the comments readable

P.S. I am going to resolve the conversation to make the comments readable
modeCh chan struct{}
// modeDoneOnce guarantees that modeCh has been closed only once.
// This may happen if SetMode has been invoked many times.
modeDoneOnce sync.Once
// closeCh is channel to indicate Close has been invoked and
// counting must be cancelled.
closeCh chan struct{}
aarifullin marked this conversation as resolved Outdated

Again, probably not a good idea: it is easy to miss that "give me the object number" can be blocked by some uncancellable process iterated through the whole database.

Again, probably not a good idea: it is easy to miss that "give me the object number" can be blocked by some uncancellable process iterated through the whole database.

Hence DB invocation cannot be blocked (I've removed keysCountDone) but it returns 0 if the value has not been initilized yet. I think returning 0 is the best option and better than returning error ErrNotInitYet

Hence `DB` invocation cannot be blocked (I've removed `keysCountDone`) but it returns 0 if the value has not been initilized yet. I think returning 0 is the best option and better than returning error `ErrNotInitYet`

P.S. I am going to resolve the conversation to make the comments readable

P.S. I am going to resolve the conversation to make the comments readable
// closeDoneOnce guarantees that modeCh has been closed only once.
// This may happen if Close has been invoked many times.
closeDoneOnce sync.Once
}
func (c *countAsync) cancelByClose() {
c.closeDoneOnce.Do(func() {
close(c.closeCh)
})
}
func (c *countAsync) cancelBySetMode() {
c.modeDoneOnce.Do(func() {
close(c.modeCh)
})
}
func (c *countAsync) addAndSetAsInit(value uint64) {
c.value.Add(value)
c.initMtx.Lock()
defer c.initMtx.Unlock()
aarifullin marked this conversation as resolved Outdated

Do we want auxiliary info receival affecting datapath metrics?

Do we want auxiliary info receival affecting datapath metrics?

Sorry, what is the focus of this comment at this line?
You don't like the idea that metrics are counter within getRawKey?

Sorry, what is the focus of this comment at this line? You don't like the idea that metrics are counter within `getRawKey`?

I mean that getRawKey contains metrics and traces, but what we get here is not user data we store.

I mean that `getRawKey` contains metrics and traces, but what we get here is not user data we store.

I can agree about metrics but I would like to to offer a compromise option and make getRawKey trace writecache.getRawKey event but not calculate user metrics. WDYT?

I can agree about metrics but I would like to to offer a compromise option and make `getRawKey` trace `writecache.getRawKey` event but not calculate user metrics. WDYT?

Seems ok, but why bother with traces? We have context.Background() so it will be just a small trace unrelated to anything.

Seems ok, but why bother with traces? We have `context.Background()` so it will be just a small trace unrelated to anything.

Alright. Initially I misunderstood how StartSpanFromContext and now I got the point that it does not make sense to pass context until initCounter <- Open .... recieve context. So, the refactoring is needed at first. I will create an issue and getRawKey and deleteRawKey no longer have ctx parameter (for a while)

Alright. Initially I misunderstood how `StartSpanFromContext` and now I got the point that it does not make sense to pass context until `initCounter <- Open ....` recieve context. So, the refactoring is needed at first. I will create an issue and `getRawKey` and `deleteRawKey` no longer have `ctx` parameter (for a while)

P.S. I am going to resolve the conversation to make the comments readable

P.S. I am going to resolve the conversation to make the comments readable

P.S. I am going to resolve the conversation to make the comments readable

P.S. I am going to resolve the conversation to make the comments readable
c.init = true
dstepanov-yadro marked this conversation as resolved Outdated

Here the function returns, but metrics and objCounter is not updated.

Here the function returns, but metrics and objCounter is not updated.

Right

Right

But is it correct? Please explain, I probably don't understand the logic.

But is it correct? Please explain, I probably don't understand the logic.

"Right" - I meant "you are right, I forgot to calculate metrics and objCounter" :)
I fixed this - I introduced deleteKey method that deletes key that is raw-represented. It makes trace spanning and calculate metrics

"Right" - I meant "you are right, I forgot to calculate metrics and objCounter" :) I fixed this - I introduced `deleteKey` method that deletes key that is raw-represented. It makes trace spanning and calculate metrics
}
func (c *countAsync) isInitialized() bool {
c.initMtx.RLock()
defer c.initMtx.RUnlock()
return c.init
}
func (c *countAsync) isMutable() bool {
c.mutMtx.RLock()
defer c.mutMtx.RUnlock()
return c.mutable
}
func (c *countAsync) setAsMutable() {
c.mutMtx.Lock()
defer c.mutMtx.Unlock()
c.mutable = true
}
func (c *countAsync) reset() {
c.value.Store(0)
c.closeCh = make(chan struct{})
c.closeDoneOnce = sync.Once{}
c.modeCh = make(chan struct{})
c.modeDoneOnce = sync.Once{}
c.initMtx.Lock()
defer c.initMtx.Unlock()
c.init = false
c.mutMtx.Lock()
defer c.mutMtx.Unlock()
c.mutable = false
}
type counters struct { type counters struct {
cDB atomic.Uint64 cDB countAsync
} }
func (x *counters) IncDB() { func (x *counters) IncDB() {
x.cDB.Add(1) if x.cDB.isMutable() {
x.cDB.value.Inc()
}
} }
func (x *counters) DecDB() { func (x *counters) DecDB() {
x.cDB.Add(math.MaxUint64) if x.cDB.isMutable() {
x.cDB.value.Dec()
}
} }
func (x *counters) DB() uint64 { func (x *counters) DB() uint64 {
return x.cDB.Load() if x.cDB.isInitialized() {
return x.cDB.value.Load()
}
return 0
}
func (x *counters) isReadyToFlush() bool {
return x.cDB.isInitialized()
} }
func (c *cache) initCounters() error { func (c *cache) initCounters() error {
c.objCounters.cDB.reset()
c.initCounterWG.Add(1)
go func() {
defer func() {
c.initCounterWG.Done()
}()
var inDB uint64 var inDB uint64
err := c.db.View(func(tx *badger.Txn) error {
if !c.options.recountKeys {
k := keyCountPrefix
cDB, err := c.getRawKey(k[:])
if err == nil {
c.objCounters.cDB.setAsMutable()
inDB = binary.LittleEndian.Uint64(cDB)
c.modeMtx.RLock()
defer c.modeMtx.RUnlock()
if !c.mode.ReadOnly() {
if err = c.deleteRawKey(k[:]); err != nil {
c.log.Error(logs.WritecacheBadgerCouldNotDeleteKeysCount, zap.Error(err))
}
}
return
}
}
// Recount keys if "key_count" has not been found
if err := c.db.View(func(tx *badger.Txn) error {
c.objCounters.cDB.setAsMutable()

I think it still may be invalid: snapshot for View is fixed before function executes and we "set as mutable" inside a function, so some objects may possibly arrived and will not be in counters. Am I right?

I think it still may be invalid: snapshot for `View` is fixed before function executes and we "set as mutable" inside a function, so some objects may possibly arrived and will not be in counters. Am I right?
  1. We have fixed the snapshot at the T1 moment. We have N objects at T1
  2. An object arrived at T2 moment. It'll try to increase object counter objCounters and we have M objects while counting is going on the snapshot
func (x *counters) IncDB() {
	if x.cDB.isMutable() {
		x.cDB.value.Inc()
	}
}
  1. While object are being counted on the snapshot [T1, Tk], we can inc/dec objCounters. Iterators are valid within View
  2. N + M - good result

So, is something wrong?

1. We have fixed the snapshot at the `T1` moment. We have `N` objects at `T1` 2. An object arrived at `T2` moment. It'll try to increase object counter `objCounters` and we have `M` objects while counting is going on the snapshot ```golang func (x *counters) IncDB() { if x.cDB.isMutable() { x.cDB.value.Inc() } } ``` 3. While object are being counted on the snapshot `[T1, Tk]`, we can inc/dec `objCounters`. Iterators are valid within `View` 4. `N + M` - good result So, is something wrong?
opts := badger.DefaultIteratorOptions opts := badger.DefaultIteratorOptions
opts.PrefetchValues = false opts.PrefetchValues = false
it := tx.NewIterator(opts) it := tx.NewIterator(opts)
defer it.Close() defer it.Close()
for it.Rewind(); it.Valid(); it.Next() { for it.Rewind(); it.Valid(); it.Next() {
select {
case <-c.objCounters.cDB.closeCh:
return errCountCancelled
case <-c.objCounters.cDB.modeCh:
return errCountCancelled
default:
inDB++ inDB++
} }
}
return nil return nil
}) }); err != nil {
if err != nil { c.log.Error(logs.WritecacheBadgerCouldNotCountKeys, zap.Error(err))
return fmt.Errorf("could not read write-cache DB counter: %w", err) return
} }
c.objCounters.cDB.Store(inDB)
c.metrics.SetActualCounters(inDB, 0) c.metrics.SetActualCounters(inDB, 0)
c.objCounters.cDB.addAndSetAsInit(inDB)
}()
return nil return nil
} }

View file

@ -21,6 +21,8 @@ type store struct {
db *badger.DB db *badger.DB
} }
var keyCountPrefix = []byte("keys_count")
type internalKey [len(cid.ID{}) + len(oid.ID{})]byte type internalKey [len(cid.ID{}) + len(oid.ID{})]byte

@fyrchik Here is different type and length for keys_count key. It seems it doesn't break anything (I hope so)

@fyrchik Here is different type and length for `keys_count` key. It seems it doesn't break anything (I hope so)

Now why 20? Isn't it just len("keys_count") == 10?

Now why 20? Isn't it just `len("keys_count") == 10`?

I mean we can have global var keyKey = []byte("keys_count") without any functions (name can be changed)

I mean we can have global `var keyKey = []byte("keys_count")` without any functions (name can be changed)
func (k internalKey) address() oid.Address { func (k internalKey) address() oid.Address {