`log.With` is suitable during initialization, but in other places it induces
some overhead, even when branches with logging are not taken.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
If we should process address based on some condition, there is no need
to read file content in memory.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
Currently we use `(*bbolt.Bucket).Stats().KeyN` for estimating database
size. However, it iterates over all pages in bucket and thus heavily
depends on the bucket size. This commit replaces initial size estimation
with a single `os.Stat` call.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
Core changes:
* avoid package-colliding variable naming
* avoid using pointers to IDs where unnecessary
* avoid using `idSDK` import alias pattern
* use `EncodeToString` for protocol string calculation and `String` for
printing
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
- Delete objects physically on tombstone's arrival;
- Store information about tombstones in the Graveyard;
- Clear Graveyard every epoch based on the information about TS in the
network.
Signed-off-by: Pavel Karpy <carpawell@nspcc.ru>
Add offset element to the iterations over deleted objects (both the
Graveyard and the Garbage buckets).
Signed-off-by: Pavel Karpy <carpawell@nspcc.ru>
It allows storing information about object in both ways at the same time:
1. Metabase should know if an object is covered by a tombstone (that is
not expired yet);
2. It should be possible to physically delete objects covered by a
tombstone immediately (mark with GC) but keep tombstone knowledge.
Signed-off-by: Pavel Karpy <carpawell@nspcc.ru>
Morph "NewEpoch" event handling was registered in a closure over
`addNewEpochNotificationHandler` func. That may lead to the data race:
if a shard was initialized before the event registration, everything works
as planned, but if registration was made earlier, it was not able to
include GC handlers since a shard has not called `eventChanInit` yet and,
therefore, it has not registered handler yet.
Signed-off-by: Pavel Karpy <carpawell@nspcc.ru>
Also, remove optimization comments:
1. Having to maintain an execute the same logic for headers as for
objects is quite inefficient, as it increases memory footprint.
2. Unmarshaling object is a cheap operation if data slice is in memory.
3. For unmarshaling header-only, I think we need SDK support.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
`Degraded` mode is set automatically after error counter is over the
threshold. `ReadOnly` mode can still be set by an administrator.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
`Batch` can execute the function multiple times leading to multiple
increases of a size approximation.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
`apistatus` package provides types which implement build-in `error`
interface. Add `error of type` pattern when documenting these errors in
order to clarify how these errors should be handled (e.g. `errors.Is` is
not good).
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Replace `ErrNotFound`/`ErrAlreadyRemoved` error from
`pkg/core/object` package with `ObjectNotFound`/`ObjectAlreadyRemoved`
one from `apistatus` package. These errors are returned by storage
node's server as NeoFS API statuses.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
There is a need to process expired `LOCK` objects similar to `TOMBSTONE`
ones: we collect them on `Shard`, notify all other shards about
expiration so they could unlock the objects, and only after that mark
lockers as garbage.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Make `FormatValidator.ValidateContent` to verify payload of `LOCK`
objects. Pass locked objects to `Locker` interface. Require from
`Locker.Lock` to return `apistatus.IrregularObjectLock` error on a
corresponding condition.
Also add error return to `DeleteHandler.DeleteObjects` method. Require
from method to return `apistatus.ObjectLocked` error on a corresponding
condition. Adopt implementations.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
`Inhume` operation can potentially mark lockers as garbage. There is a
need to update locker list in locked bucket.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Make `StorageEngine.Delete` to forward first encountered
`apistatus.ObjectLocked` error during shard processing.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Make `StorageEngine.Inhume` to forward first encountered
`apistatus.ObjectLocked` error during shard processing.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Implement `StorageEngine.Lock` method which works similar to `Inhume`
but calls `Lock` on the processing shards.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Make `DB.Lock` to return `apistatus.IrregularObjectLock` if at least one
of the locked objects is irregular (not of type REGULAR).
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Make `DB.Inhume` to return `apistatus.ObjectLocked` if at least one of
the inhumed objects is locked.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Make `DB.IterateCoveredByTombstones` to not pass locked objects to the
handler. The method is used by GC, therefore it will not consider locked
objects as candidates for deletion even if their tombstone is expired.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Make `DB.IterateExpired` to not pass locked objects to the handler. The
method is used by GC, therefore it will not consider them as candidates
for deletion.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
After introduction of LOCK objects (of type `TypeLock`) complicated
extended its behavior:
* create `lockers` container bucket (LCB) during PUT;
* remove object from LCB during DELETE;
* look up object in LCB during EXISTS;
* get object from LCB during GET;
* list objects from LCB during LIST with cursor;
* select objects from LCB during SELECT with '*'.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Implement `DB.Lock` method which marks list of the objects as locked by
another object. Only regular objects can be locked.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Create class of container buckets with `LOCKED` suffix. Put identifiers
of the objects of type `LOCK` to these buckets during `DB.Put`.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Remove `Object` and `RawObject` types from `pkg/core/object` package.
Use `Object` type from NeoFS SDK Go library everywhere. Avoid using the
deprecated elements.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Similarly to `Get`. Also fix a bug where `ErrNotFound` is returned
instead of `ErrRangeOutOfBounds`.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
Metabase is expected to contain actual information about objects stored
in shard. If the object is present in metabase but is missing from
blobstor, peform an additional attempt to fetch it directly without
consulting metabase. Such a situation is unexpected, so error counter
is increased for the shard which has the object in the metabase. We
don't increase error counter for the shard which has the object in
blobstor, because some garbage can be expected there. In this
implementation there is no overhead for objects which are really
missing, i.e. are not present in any metabase.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
There are certain errors which are not expected during usual node
operation and which tell us that something is wrong with the shard.
To prevent possible data corruption, move shard in read-only mode after
amount of errors exceeded some threshold. By default no actions are performed.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
If pre-existing blobovnicza is initialized, it's size should be updated
even if all buckets are in place.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
It was added back in 2fb379b7 when we had many shard modes. Now we have
only two and comments for constants are rather descriptive.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
We could also ignore errors during evacuate, but this requires
unmarshaling objects first which slowers the process considerably.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
In read-only mode modifying operations are immediately returned with
error and all background operations are suspended.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
Dump contains magic and a list of objects prefixed by object size in bytes.
We can't use proto-marshaled list because this requires having all dump
in memory. Using TAR induces 512 byte overhead for each object which can
be a problem in some cases.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
For some data compression makes little sense, as it is already compressed.
This commit allows to leave such data unchanged based on `Content-Type`
attribute. Currently exact, prefix and suffix matching are supported.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
Provide shard mode information via `DumpInfo()`. Delete atomic field from
Shard structure since it duplicates new field.
Signed-off-by: Pavel Karpy <carpawell@nspcc.ru>
Shard's mode was not used in the Node, so added only two modes whose roles
are clear. More modes will be added in the future.
Signed-off-by: Pavel Karpy <carpawell@nspcc.ru>
Make `flushBigObjects` routine to mark objects which are written to
`BlobStor`. This prevents already flushed objects from being written on
the next iterator tick.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
For fullness estimation of `Blobovnicza` we use number of object stored
in each size bucket. In previous implementation we multiplied the number
by the difference in bucket boundaries. This expression rather
estimated the minimum volume (and for the smallest bucket, the maximum)
of objects in the bucket.
Multiply number of objects by mean bucket size.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Make `syncFullnessCounter` to accept `bbolt.Tx` argument of Bolt
transaction within which counter should be synchronized. Pass
corresponding transaction during `Init`.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
According to BoltDB documentation bucket `value is only valid for the
life of the transaction`.
Make `DB.IsSmall` copy value slice in order to prevent potential memory
corruptions (e.g. `runtime.stringtobyteslice` cast).
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
ListWithCursor allows listing physically stored objects
from metabase with small chunks. Cursor tracks last
processed object, therefore new chunks are returned
on each request.
Signed-off-by: Alex Vanin <alexey@nspcc.ru>
Make `BlockExecution` / `ResumeExecution` to not release per-shard worker
pools. Make `StorageEngine.Close` to block these methods and any
data-related operations. It is still releases the pools.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Use `sync.Once` to prevent locks of stopping GC. It will also allow to
safely call `Shard.Close` multiple times.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
There is a need to disable execution of local data operation on storage
engine in runtime. If storage engine ops are blocked, node will act like
always but all local object operations will be denied.
Implement `BlockExecution` / `ResumeExecution` methods on `StorageEngine`
which blocks / resumes the execution of data ops. Wait for the completion of
all operations executed at the time of the call. Return error passed to
`BlockExecution` from all data-related methods until `ResumeExecution` call.
Make `Close` to block operations as well.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
a1696a8 introduced some logic which in some situations prevented big objects
to be persisted in FSTree. In this commit a refactoring is done with the
goal of simplifying the code and also checking #866 issue.
1. Split a monstrous function into multiple simple ones: memory objects
can only be small and for writing through the cache we can do a dispatch
in `Put` itself.
2. Determine objects to be put in database before the actual update
as setting up a transaction has non-zero overhead.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
Container listing should not ignore tombstone and
storage group objects which are not stored in
primary buckets.
Signed-off-by: Alex Vanin <alexey@nspcc.ru>
Some of the pools are initialized during config initialization,
so it isn't possible currently to release them in one place.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
Make `StorageEngine` to use non-blocking worker pools with the same
(configurable) size for PUT operation. This allows you to switch to using
more free shards when overloading others, thereby more evenly distributing
the write load.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
We should be able to read whatever we have written earlier.
Compression setting applies only to the new objects.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
Do not log in options constructors. Also failure to
initialize compression module (possibly due to invalid options) is
certainly an error deserving proper treatment.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
There is a need to list addresses of the small objects stored in WriteCache
database.
Implement `IterateDB` function which accepts BoltDB instance and iterate
over all saved objects and passes their addresses to the hander.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
There is a need to open Blobovnicza instances in read-only mode in some
cases.
Add `ReadOnly` option. Do not create dir path in RO. Open underlying BoltDB
instance with ReadOnly flag. Document thal all writing operations should not
be called in ro (otherwise BoltDB txs fail).
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
`Blobovnicza` can be initialized with any number of range buckets, and
reconstructed with different size limit. In previous implementation
`Iterate` could miss some stored objects if we construct `Blobovnicza` with
smaller number of ranges.
Make `Iterate` to traverse all buckets regardless of current instance
bounds.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
In previous implementation `Blobovnicza.Iterate` op decoded object data only
and passed it to the handler. There is a need to iterate over all addresses
of the stored objects.
Add `DecodeAddresses` and `WithoutData` methods of `IteratePrm` type. Add
`Address` method to `IterationElement` type. Make `Iterate` to decode object
addresses if `DecodeAddress` was called and not read the data if
`WithoutData` was called. Implement `IterateAddresses` helper function to
simplify the code.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Each object from graveyard has tombstone or GC mark. If object has
tombstone, metabase should return `ErrAlreadyRemoved` on object requests.
This is the case when user clearly removed the object from container. GC
marks are used for physical removal which can appear even if object is still
presented in container (Control service, Policer job, etc.). In this case
metabase should return 404 error on object requests.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
`List` method of `Shard` must return only physically stored objects.
Use `AddPhyFilter` to select only phy objects.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Tombstone and "alive" objects can be both stored in BlobStor. They can
appear during iterating in different order. Metabase returns
`ErrAlreadyRemoved` error if object is inhumed.
Ignore `object.ErrAlreadyRemoved` errors of `metabase.Put`in Shard's
`refillMetabase` operation.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
There is a need to refill Metabase data with the objects from BlobStor.
Implement `refillMetabase` method which iterates over all objects from
BlobStor and saves them in Metabase.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
There is a need to be able to process all objects saved in `BlobStor`.
Implement `BlobStor.Iterate` method which iterates over all objects.
Implement `IterateBinaryObjects` and `IterateObjects` helper functions to
simplify the code.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
There is a need to be able to process all stored objects saved in
`Blobovnicza`.
Implement `Blobovnicza.Iterate` method which iterates over all objects.
Implement `IterateObjects` helper function to simplify the code.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
In the previous implementation of the metabase, there was no possibility of
reinitializing the metabase: clearing information about existing objects and
bringing it back to its initial state. This operation can be useful in
cases when the stored metadata about objects has lost (or possibly lost)
relevance, and you need to generate data from scratch. Also at the
initialization stage, static resources of the base were not created -
container-independent buckets.
Make `Metabase.Init` method to allocate graveyard, container-size and
to-move-it buckets in underlying BoltDB instance. Implement `Metabase.Reset`
method: it works like `Init` but clean up all static buckets and removes
other ones. Due to the logical similarity, the methods share a single piece
of code.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
There is a need to limit disk space used by write-cache. It is almost
impossible to calculate the value exactly. It is proposed to estimate the
size of the cache by the number of objects stored in it.
Track amounts of objects saved in DB and FSTree separately. To do this,
`ObjectCounters` interface is defined. It is generalized to a store of
numbers that can be made persistent (new option `WithObjectCounters`). By
default DB number is calculated as key number in default bucket, and FS
number is set same to DB since it is currently hard to read the actual value
from `FSTree` instance. Each PUT/DELETE operation to DB or FS
increases/decreases corresponding counter. Before each PUT op an overflow
check is performed with the following formula for evaluating the occupied
space: `NumDB * MaxDBSize + NumFS * MaxFSSize`. If next PUT can cause
write-cache overflow, object is written to the main storage.
By default maximum write-cache size is set to 1GB.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
There is a need to keep track of each local storage change. Log messages are
the most convenient way to do it.
Implement function which writes log message about the completed writing
operation in storage engine.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Shard should try to read object headers from write-cache if it is enabled.
Extend `writecache.Cache` interface with `Head` method. Call the method in
`Shard.Head` if `Shard.hasWriteCache` returns true.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Write cache should be able to execute HEAD operations according to spec.
Add simple implementation of `Head` method through the `Get` one. Leave
notes for future optimization.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Changes:
* replace `iotuil` elements with the ones from `os` package;
* replace `os.Filemode` with `fs.FileMode`;
* use `signal.NotifyContext` instead of `NewGracefulContext` (removed).
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
If object to be inhumed is root we need to continue first traverse over the
shards. In case when several children are stored in different shards,
inhuming object in a single shard leads to appearance of inhumed object in
subsequent selections. Also, any object can be already inhumed, and this
case is equivalent to successful inhume.
Do not fail on `object.ErrAlreadyRemoved` error. Continue first iterating
over shards if we detected root object (`SplitInfoError`).
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Write unit tests of `StorageEngine.Inhume` which assert that inhumed objects
don't appear in `Select` result.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Evicting from cache requires closing blobovnicza which
in turn needs to lock `activeMtx`. This lock is not needed on
every addition, but our LRU library doesn't return evicted keys.
In future we may consider switching to other implementation.
Signed-off-by: Evgenii Stratonikov <evgeniy@nspcc.ru>
This function already reused in different storage engine parts
so it makes sense to keep it in separate package.
Signed-off-by: Alex Vanin <alexey@nspcc.ru>
Different SplitInfo parts may be stored in different shards. Storage
engine must not stop at first SplitInfoError and should make
best effort to complete SplitInfo structure if needed.
Signed-off-by: Alex Vanin <alexey@nspcc.ru>
There were no unit tests of storage engine. This commit
adds first test to reproduce missing link ID in split info
at `engine.Head(raw)` request.
Engine tests uses some constructors from metabase tests,
so it is better to locate such functions in common
package at local_object_storage.
Signed-off-by: Alex Vanin <alexey@nspcc.ru>
`Inhume` operation can be performed on already deleted objects, and in this
case the entry will be added to the graveyard. `Delete` operation finishes
with error if object is not presented in metabase. However, the entry in the
cemetery must be deleted regardless of the presence of the object.
Additionally, now `Delete` does not return an error in the absence of an
object.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Metabase should not store payloads of objects. Make Put operation to cut
object payload before saving binary object in metabase.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Metabase should not store payloads of objects. Set payload in generated test
object. Ascertain that objects returned by Get method have no payload.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Add `InhumePrm.MarkAsGarbage` method which marks passed objects to be
removed from local storage. Update `InhumePrm.WithTarget` doc to prevent
conflicting use with the new method.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Creating tombstones for tombstones is prohibited in NeoFS system. Metabase
graveyard contains records of the form {address: address}: key is an address
of inhumed object, value is an address of the tombstone. To prevent creation
tombstones for tombstones metabase must control incoming Inhume calls:
* if Inhume target is a tombstone, then "grave" should not be added;
* if {a1:a2} "grave" was created earlier and {a2: a3} "grave" came later,
then first "grave" must be removed as tomb-on-tomb.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Change Shard's garbage remover to interrupt iterating over the metabase
graveyard when the buffer is full to the max size (`WithRemoverBatchSize`
Shard's option).
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Make `DB.IterateOverGraveyard` to immediately return nil if GraveHandler
returns ErrInterruptIterator.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Add new epoch event handler to GC that finds all expired tombstones and
marks them and underlying objects to be removed. Shard uses callbacks
provided by the storage engine to mark underlying objects.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Implement `DB.IterateCoveredByTombstones` method that iterates over graves
and handles all objects under one of the tombstones.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Add new epoch event handler to GC that finds all expired non-tombstone
objects and marks them to be removed.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Implement `DB.IterateExpired` method that iterates over the objects in
metabase that are expired at particular epoch.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Group handlers of the particular event to a WaitGroup and wait for it before
the next event handling. This will ensure that all handlers complete and
prevent potential conflicts between past and present jobs.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
`Shard.Init` method creates a new GC instance from shard configuration and
starts GC's workers through `init` call. In initial implementation GC
routines are indefinite and can be killed only with by application shutdown.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Shard's GC component consists of:
* asynchronous remover that periodically wake up and removes all garbage
objects from the shard, and goes to sleep for particular time interval;
* external event listener that distributes jobs between workers;
* group of workers that can handle a single job related to particular
external event.
Remover and event listener represents go-routines which are started by
`init` method (calls from `Shard.Init`). In initial version all event
handlers are interrupted: this means that next event of the same type will
interrupt previous handling and start the new one.
GC is fully encapsulated in Shard. All GC configurations are reflected in
Shard's configuration.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Implement `DB.IterateOverGraveyard` method that iterates over all graves and
passes passes their descriptors (new type `Grave`) to handler (new type
`GraveHandler`). `Grave` currently have buried object address and garbage
flag.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Replace single target address in `InhumePrm` with the list of addresses.
Change corresponding parameter in `WithTarget` and `MarkAsGarbage` methods
to variadic.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Replace single target address in `InhumePrm` with the list of addresses.
Rename `WithAddress` method to `WithAddresses` and change parameter to
variadic.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Make `StorageEngine.Delete` to execute `Inhume` operation with
`MarkAsGarbage` parameter on the `Shard` that holds the object. Searching of
the particular shard is performed through iterating over HRW-sorted shards.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Implement `InhumePrm.MarkAsGarbage` method that leads to marking object as
garbage in metabase. Update `InhumePrm.WithTarget` doc indicating a conflict
with the new method.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Implement `InhumePrm.WithGCMark` method that marks the object as garbage in
graveyard. Update `InhumePrm.WithTombstoneAddress` doc indicating a conflict
with the new method. Update `Inhume` function doc about tombstone address
parameter.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Delete operation of Metabase is performed on group of objects. The set being
removed can contain descendants of a common parent. In the case when all
descendants of a parent object are deleted, it must also be deleted from
the metabase. In the previous implementation, this was not done due to the
chosen approach to counting references to the parent.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
In previous implementation StorageEngine.Inhume operation forced Shard
.Inhume call on all internal shards. There is a need to inhume object in a
single shard. To achieve this, Inhume operation is performed in next steps:
1. iterate over sorted shards, check object presence through Exists call;
2. if object exists at any shard in step 1 => inhume it and return on
success;
3. if no shards contain the object => iterate over sorted shards again and
try to inhume the object at first possible shard;
4. if all Inhume calls are failed => return an error.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Container listing already supported in the metabase for `engine.List`
operation. To get container statistics engine should provide both the
option to get container volume estimation and list of all containers.
Signed-off-by: Alex Vanin <alexey@nspcc.ru>
Objects of one container can be split among shards, so engine
should iterate over all available shards to sum all size
estimations.
Signed-off-by: Alex Vanin <alexey@nspcc.ru>
Storage nodes keep container size estimation so they
can announce this info and hope for some basic income
settlements. This is also useful for monitoring.
Container size does not include non regular or inhumed
object sizes.
Signed-off-by: Alex Vanin <alexey@nspcc.ru>
There is a codecov issue because objects are not placed
in the engine the same way every unit test. Therefore
sometimes there are more coverage, sometimes there are
less. Seeded RNG should solve this issue for engine tests.
Signed-off-by: Alex Vanin <alexey@nspcc.ru>
In previous implementation DB.Containers method could return an error about
invalid container ID string format. This could happen if some of top-level
buckets had name w/o "_" substring.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Deadlock occurs when `getActivate` function opens new blobovnicza and that
invokes evict in LRU cache of open blobovniczas. `getActivate` makes
`activeMtx.Lock()` and then cache evict makes `activeMtx.RLock()` and deadlock
happens.
Fix contains two steps:
- add separate mutex to open blobovniczas (1),
- split single Lock outside of `updateAndGet` (2).
As for the (1) `bbolt.Open()` locks when it tries to open the same file from
two threads. So separate mutex will prevent that.
As for the (2) `updateAndGet` function contains from two parts. At first it
checks if required blobovnicza is ready and it returns it. In this case we can
use the simple RLock. But then there is an option when we should open new
blobovnicza and update map of active blobovniczas.
In this case we call `openBlobovnicza` without activeMtx lock. Cache evict
happens there and it won't cause deadlock.
Then we lock activeMtx to update the map of active blobovniczas. Concurrency can
happen there. However `openBlobovnicza` will not open the same blobovnicza twice,
so we can make one more check if opened blobovnicza was activated while thread was
locked in activeMtx. If so, then return active blobovnicza, else finish activation.
Signed-off-by: Alex Vanin <alexey@nspcc.ru>
In previous implementation Blobovnicza could incorrectly initialize
dimensional buckets: if SmallSizeLimit = 2 ^ X + Y && Y < 2 ^ X, then
largest dimensional bucket was [2 ^ (X - 1) : 2 ^ X]. This was caused by an
incorrect condition for stopping the iterator along the dimensional
boundaries.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
All parameters and resulting values of all metabase operations are
structured in new types. The most popular scenarios for using operations are
moved to auxiliary functions.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
In previous implementation Blobovnicza's stored objects in protocol format
which did not allow working with externally compressed objects. To achieve
this goal, operations Get and Put no longer work with the structure of the
object, but only with abstract binary data. Operation GetRange has become
incorrect in its original purpose to receive the payload range. In this
regard, BlobStor receives the payload range of the object through Get
operation. In the future either Blobovnicza will learn to compress objects
by itself, or the GetRange operation will be eliminated.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Blobovnicza returns object, so we can't put compressed
data there. Compressed data won't be deserialized correctly.
Signed-off-by: Alex Vanin <alexey@nspcc.ru>
Replace ErrNotFound and ErrRangeOutOfBounds to core/object package in order
to share them across the libraries.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
Add blobovnicza instance to BlobStor structure. Create blobovnicza tree in
BlobStor constructor. Implement Open/Init/Close methods.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>
There is a need to support single blobovnicza in blobovnicza tree. This can
be achieved with a width of 1, and a depth of 0 or 1. With depth = 1 one
redundant directory is created, inside which there is a blobovnicza. If the
depth is zero, the blobobnivza will be in the root path. Fix negative
capacity in iterateDeepest method with zero depth.
Signed-off-by: Leonard Lyubich <leonard@nspcc.ru>